-
-
Notifications
You must be signed in to change notification settings - Fork 149
/
core.py
742 lines (631 loc) · 25.9 KB
/
core.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
import asyncio
import copy
import getpass
import logging
import os
import string
import time
import uuid
from weakref import finalize
try:
import yaml
except ImportError:
yaml = False
import dask
import dask.distributed
import distributed.security
from distributed.deploy import SpecCluster, ProcessInterface
from distributed.utils import Log, Logs
import kubernetes_asyncio as kubernetes
from kubernetes_asyncio.client.rest import ApiException
from .objects import (
make_pod_from_dict,
make_service_from_dict,
clean_pod_template,
clean_service_template,
)
from .auth import ClusterAuth
logger = logging.getLogger(__name__)
SCHEDULER_PORT = 8786
class Pod(ProcessInterface):
""" A superclass for Kubernetes Pods
See Also
--------
Worker
Scheduler
"""
def __init__(self, core_api, pod_template, namespace, loop=None, **kwargs):
self._pod = None
self.core_api = core_api
self.pod_template = copy.deepcopy(pod_template)
self.base_labels = self.pod_template.metadata.labels
self.namespace = namespace
self.name = None
self.loop = loop
self.kwargs = kwargs
super().__init__()
@property
def cluster_name(self):
return self.pod_template.metadata.labels["dask.org/cluster-name"]
async def start(self, **kwargs):
retry_count = 0 # Retry 10 times
while True:
try:
self._pod = await self.core_api.create_namespaced_pod(
self.namespace, self.pod_template
)
return await super().start(**kwargs)
except ApiException as e:
if retry_count < 10:
logger.debug("Error when creating pod, retrying... - %s", str(e))
await asyncio.sleep(1)
retry_count += 1
else:
raise e
async def close(self, **kwargs):
name, namespace = self._pod.metadata.name, self.namespace
if self._pod:
try:
await self.core_api.delete_namespaced_pod(name, namespace)
except ApiException as e:
if e.reason == "Not Found":
logger.debug(
"Pod %s in namespace %s has been deleated already.",
name,
namespace,
)
else:
raise
await super().close(**kwargs)
async def logs(self):
try:
log = await self.core_api.read_namespaced_pod_log(
self._pod.metadata.name, self.namespace
)
except ApiException as e:
if "waiting to start" in str(e):
log = ""
else:
raise e
return Log(log)
async def describe_pod(self):
self._pod = await self.core_api.read_namespaced_pod(
self._pod.metadata.name, self.namespace
)
return self._pod
def __repr__(self):
return "<Pod %s: status=%s>" % (type(self).__name__, self.status)
class Worker(Pod):
""" A Remote Dask Worker controled by Kubernetes
Parameters
----------
scheduler: str
The address of the scheduler
name (optional):
The name passed to the dask-worker CLI at creation time.
"""
def __init__(self, scheduler: str, name=None, **kwargs):
super().__init__(**kwargs)
self.scheduler = scheduler
self.pod_template.metadata.labels["dask.org/component"] = "worker"
self.pod_template.spec.containers[0].env.append(
kubernetes.client.V1EnvVar(
name="DASK_SCHEDULER_ADDRESS", value=self.scheduler
)
)
if name is not None:
worker_name_args = ["--name", str(name)]
self.pod_template.spec.containers[0].args += worker_name_args
class Scheduler(Pod):
""" A Remote Dask Scheduler controled by Kubernetes
Parameters
----------
idle_timeout: str, optional
The scheduler task will exit after this amount of time
if there are no requests from the client. Default is to
never timeout.
service_wait_timeout_s: int (optional)
Timeout, in seconds, to wait for the remote scheduler service to be ready.
Defaults to 30 seconds.
Set to 0 to disable the timeout (not recommended).
"""
def __init__(self, idle_timeout: str, service_wait_timeout_s: int = None, **kwargs):
super().__init__(**kwargs)
self.service = None
self._idle_timeout = idle_timeout
self._service_wait_timeout_s = service_wait_timeout_s
if self._idle_timeout is not None:
self.pod_template.spec.containers[0].args += [
"--idle-timeout",
self._idle_timeout,
]
async def start(self, **kwargs):
await super().start(**kwargs)
while (await self.describe_pod()).status.phase == "Pending":
await asyncio.sleep(0.1)
while self.address is None:
logs = await self.logs()
for line in logs.splitlines():
if "Scheduler at:" in line:
self.address = line.split("Scheduler at:")[1].strip()
await asyncio.sleep(0.1)
self.service = await self._create_service()
self.address = "tcp://{name}.{namespace}:{port}".format(
name=self.service.metadata.name,
namespace=self.namespace,
port=SCHEDULER_PORT,
)
if self.service.spec.type == "LoadBalancer":
# Wait for load balancer to be assigned
start = time.time()
while self.service.status.load_balancer.ingress is None:
if (
self._service_wait_timeout_s > 0
and time.time() > start + self._service_wait_timeout_s
):
raise asyncio.TimeoutError(
"Timed out waiting for Load Balancer to be provisioned."
)
self.service = await self.core_api.read_namespaced_service(
self.cluster_name, self.namespace
)
await asyncio.sleep(0.2)
[loadbalancer_ingress] = self.service.status.load_balancer.ingress
loadbalancer_host = loadbalancer_ingress.hostname or loadbalancer_ingress.ip
self.external_address = "tcp://{host}:{port}".format(
host=loadbalancer_host, port=SCHEDULER_PORT
)
# FIXME Set external address when using nodeport service type
# FIXME Create an optional Ingress just in case folks want to configure one
async def close(self, **kwargs):
if self.service:
await self.core_api.delete_namespaced_service(
self.cluster_name, self.namespace
)
await super().close(**kwargs)
async def _create_service(self):
service_template_dict = dask.config.get("kubernetes.scheduler-service-template")
self.service_template = clean_service_template(
make_service_from_dict(service_template_dict)
)
self.service_template.metadata.name = self.cluster_name
self.service_template.metadata.labels = copy.deepcopy(self.base_labels)
self.service_template.spec.selector["dask.org/cluster-name"] = self.cluster_name
if self.service_template.spec.type is None:
self.service_template.spec.type = dask.config.get(
"kubernetes.scheduler-service-type"
)
await self.core_api.create_namespaced_service(
self.namespace, self.service_template
)
return await self.core_api.read_namespaced_service(
self.cluster_name, self.namespace
)
class KubeCluster(SpecCluster):
""" Launch a Dask cluster on Kubernetes
This starts a local Dask scheduler and then dynamically launches
Dask workers on a Kubernetes cluster. The Kubernetes cluster is taken
to be either the current one on which this code is running, or as a
fallback, the default one configured in a kubeconfig file.
**Environments**
Your worker pod image should have a similar environment to your local
environment, including versions of Python, dask, cloudpickle, and any
libraries that you may wish to use (like NumPy, Pandas, or Scikit-Learn).
See examples below for suggestions on how to manage and check for this.
**Network**
Since the Dask scheduler is launched locally, for it to work, we need to
be able to open network connections between this local node and all the
workers nodes on the Kubernetes cluster. If the current process is not
already on a Kubernetes node, some network configuration will likely be
required to make this work.
**Resources**
Your Kubernetes resource limits and requests should match the
``--memory-limit`` and ``--nthreads`` parameters given to the
``dask-worker`` command.
Parameters
----------
pod_template: kubernetes.client.V1Pod
A Kubernetes specification for a Pod for a dask worker.
scheduler_pod_template: kubernetes.client.V1Pod (optional)
A Kubernetes specification for a Pod for a dask scheduler.
Defaults to the pod_template.
name: str (optional)
Name given to the pods. Defaults to ``dask-$USER-random``
namespace: str (optional)
Namespace in which to launch the workers.
Defaults to current namespace if available or "default"
n_workers: int
Number of workers on initial launch.
Use ``scale`` to change this number in the future
env: Dict[str, str]
Dictionary of environment variables to pass to worker pod
host: str
Listen address for local scheduler. Defaults to 0.0.0.0
port: int
Port of local scheduler
auth: List[ClusterAuth] (optional)
Configuration methods to attempt in order. Defaults to
``[InCluster(), KubeConfig()]``.
idle_timeout: str (optional)
The scheduler task will exit after this amount of time
if there are no requests from the client. Default is to
never timeout.
scheduler_service_wait_timeout: int (optional)
Timeout, in seconds, to wait for the remote scheduler service to be ready.
Defaults to 30 seconds.
Set to 0 to disable the timeout (not recommended).
deploy_mode: str (optional)
Run the scheduler as "local" or "remote".
Defaults to ``"local"``.
**kwargs: dict
Additional keyword arguments to pass to LocalCluster
Examples
--------
>>> from dask_kubernetes import KubeCluster, make_pod_spec
>>> pod_spec = make_pod_spec(image='daskdev/dask:latest',
... memory_limit='4G', memory_request='4G',
... cpu_limit=1, cpu_request=1,
... env={'EXTRA_PIP_PACKAGES': 'fastparquet git+https://github.com/dask/distributed'})
>>> cluster = KubeCluster(pod_spec)
>>> cluster.scale(10)
You can also create clusters with worker pod specifications as dictionaries
or stored in YAML files
>>> cluster = KubeCluster.from_yaml('worker-template.yml')
>>> cluster = KubeCluster.from_dict({...})
Rather than explicitly setting a number of workers you can also ask the
cluster to allocate workers dynamically based on current workload
>>> cluster.adapt()
You can pass this cluster directly to a Dask client
>>> from dask.distributed import Client
>>> client = Client(cluster)
You can verify that your local environment matches your worker environments
by calling ``client.get_versions(check=True)``. This will raise an
informative error if versions do not match.
>>> client.get_versions(check=True)
The ``daskdev/dask`` docker images support ``EXTRA_PIP_PACKAGES``,
``EXTRA_APT_PACKAGES`` and ``EXTRA_CONDA_PACKAGES`` environment variables
to help with small adjustments to the worker environments. We recommend
the use of pip over conda in this case due to a much shorter startup time.
These environment variables can be modified directly from the KubeCluster
constructor methods using the ``env=`` keyword. You may list as many
packages as you like in a single string like the following:
>>> pip = 'pyarrow gcsfs git+https://github.com/dask/distributed'
>>> conda = '-c conda-forge scikit-learn'
>>> KubeCluster.from_yaml(..., env={'EXTRA_PIP_PACKAGES': pip,
... 'EXTRA_CONDA_PACKAGES': conda})
You can also start a KubeCluster with no arguments *if* the worker template
is specified in the Dask config files, either as a full template in
``kubernetes.worker-template`` or a path to a YAML file in
``kubernetes.worker-template-path``.
See https://docs.dask.org/en/latest/configuration.html for more
information about setting configuration values.::
$ export DASK_KUBERNETES__WORKER_TEMPLATE_PATH=worker_template.yaml
>>> cluster = KubeCluster() # automatically finds 'worker_template.yaml'
See Also
--------
KubeCluster.from_yaml
KubeCluster.from_dict
KubeCluster.adapt
"""
def __init__(
self,
pod_template=None,
name=None,
namespace=None,
n_workers=None,
host=None,
port=None,
env=None,
auth=ClusterAuth.DEFAULT,
idle_timeout=None,
deploy_mode=None,
interface=None,
protocol=None,
dashboard_address=None,
security=None,
scheduler_service_wait_timeout=None,
scheduler_pod_template=None,
**kwargs
):
self.pod_template = pod_template
self.scheduler_pod_template = scheduler_pod_template
self._generate_name = name
self._namespace = namespace
self._n_workers = n_workers
self._idle_timeout = idle_timeout
self._deploy_mode = deploy_mode
self._protocol = protocol
self._interface = interface
self._dashboard_address = dashboard_address
self._scheduler_service_wait_timeout = scheduler_service_wait_timeout
self.security = security
if self.security and not isinstance(
self.security, distributed.security.Security
):
raise RuntimeError(
"Security object is not a valid distributed.security.Security object"
)
self.host = host
self.port = port
self.env = env
self.auth = auth
self.kwargs = kwargs
super().__init__(**self.kwargs)
def _get_pod_template(self, pod_template, pod_type):
if not pod_template and dask.config.get(
"kubernetes.{}-template".format(pod_type), None
):
d = dask.config.get("kubernetes.{}-template".format(pod_type))
d = dask.config.expand_environment_variables(d)
pod_template = make_pod_from_dict(d)
if not pod_template and dask.config.get(
"kubernetes.{}-template-path".format(pod_type), None
):
import yaml
fn = dask.config.get("kubernetes.{}-template-path".format(pod_type))
fn = fn.format(**os.environ)
with open(fn) as f:
d = yaml.safe_load(f)
d = dask.config.expand_environment_variables(d)
pod_template = make_pod_from_dict(d)
return pod_template
def _fill_pod_templates(self, pod_template, pod_type):
pod_template = copy.deepcopy(pod_template)
# Default labels that can't be overwritten
pod_template.metadata.labels["dask.org/cluster-name"] = self._generate_name
pod_template.metadata.labels["dask.org/component"] = pod_type
pod_template.metadata.labels["user"] = escape(getpass.getuser())
pod_template.metadata.labels["app"] = "dask"
pod_template.metadata.namespace = self._namespace
if self.env:
pod_template.spec.containers[0].env.extend(
[
kubernetes.client.V1EnvVar(name=k, value=str(v))
for k, v in self.env.items()
]
)
pod_template.metadata.generate_name = self._generate_name
return pod_template
async def _start(self):
self._generate_name = self._generate_name or dask.config.get("kubernetes.name")
self._namespace = self._namespace or dask.config.get("kubernetes.namespace")
self._idle_timeout = self._idle_timeout or dask.config.get(
"kubernetes.idle-timeout"
)
self._scheduler_service_wait_timeout = (
self._scheduler_service_wait_timeout
or dask.config.get("kubernetes.scheduler-service-wait-timeout")
)
self._deploy_mode = self._deploy_mode or dask.config.get(
"kubernetes.deploy-mode"
)
self._n_workers = (
self._n_workers
if self._n_workers is not None
else dask.config.get("kubernetes.count.start")
)
self.host = self.host or dask.config.get("kubernetes.host")
self.port = (
self.port if self.port is not None else dask.config.get("kubernetes.port")
)
self._protocol = self._protocol or dask.config.get("kubernetes.protocol")
self._interface = self._interface or dask.config.get("kubernetes.interface")
self._dashboard_address = self._dashboard_address or dask.config.get(
"kubernetes.dashboard_address"
)
self.env = (
self.env if self.env is not None else dask.config.get("kubernetes.env")
)
self.pod_template = self._get_pod_template(self.pod_template, pod_type="worker")
self.scheduler_pod_template = self._get_pod_template(
self.scheduler_pod_template, pod_type="scheduler"
)
if not self.pod_template:
msg = (
"Worker pod specification not provided. See KubeCluster "
"docstring for ways to specify workers"
)
raise ValueError(msg)
base_pod_template = self.pod_template
self.pod_template = clean_pod_template(self.pod_template, pod_type="worker")
if not self.scheduler_pod_template:
self.scheduler_pod_template = base_pod_template
self.scheduler_pod_template.spec.containers[0].args = ["dask-scheduler"]
self.scheduler_pod_template = clean_pod_template(
self.scheduler_pod_template, pod_type="scheduler"
)
await ClusterAuth.load_first(self.auth)
self.core_api = kubernetes.client.CoreV1Api()
if self._namespace is None:
self._namespace = _namespace_default()
self._generate_name = self._generate_name.format(
user=getpass.getuser(), uuid=str(uuid.uuid4())[:10], **os.environ
)
self._generate_name = escape(self._generate_name)
self.pod_template = self._fill_pod_templates(
self.pod_template, pod_type="worker"
)
self.scheduler_pod_template = self._fill_pod_templates(
self.scheduler_pod_template, pod_type="scheduler"
)
finalize(
self, _cleanup_resources, self._namespace, self.pod_template.metadata.labels
)
common_options = {
"core_api": self.core_api,
"namespace": self._namespace,
"loop": self.loop,
}
if self._deploy_mode == "local":
self.scheduler_spec = {
"cls": dask.distributed.Scheduler,
"options": {
"protocol": self._protocol,
"interface": self._interface,
"host": self.host,
"port": self.port,
"dashboard_address": self._dashboard_address,
"security": self.security,
},
}
elif self._deploy_mode == "remote":
self.scheduler_spec = {
"cls": Scheduler,
"options": {
"idle_timeout": self._idle_timeout,
"service_wait_timeout_s": self._scheduler_service_wait_timeout,
"pod_template": self.scheduler_pod_template,
**common_options,
},
}
else:
raise RuntimeError("Unknown deploy mode %s" % self._deploy_mode)
self.new_spec = {
"cls": Worker,
"options": {"pod_template": self.pod_template, **common_options},
}
self.worker_spec = {i: self.new_spec for i in range(self._n_workers)}
await super()._start()
@classmethod
def from_dict(cls, pod_spec, **kwargs):
""" Create cluster with worker pod spec defined by Python dictionary
Examples
--------
>>> spec = {
... 'metadata': {},
... 'spec': {
... 'containers': [{
... 'args': ['dask-worker', '$(DASK_SCHEDULER_ADDRESS)',
... '--nthreads', '1',
... '--death-timeout', '60'],
... 'command': None,
... 'image': 'daskdev/dask:latest',
... 'name': 'dask-worker',
... }],
... 'restartPolicy': 'Never',
... }
... }
>>> cluster = KubeCluster.from_dict(spec, namespace='my-ns') # doctest: +SKIP
See Also
--------
KubeCluster.from_yaml
"""
return cls(make_pod_from_dict(pod_spec), **kwargs)
@classmethod
def from_yaml(cls, yaml_path, **kwargs):
""" Create cluster with worker pod spec defined by a YAML file
We can start a cluster with pods defined in an accompanying YAML file
like the following:
.. code-block:: yaml
kind: Pod
metadata:
labels:
foo: bar
baz: quux
spec:
containers:
- image: daskdev/dask:latest
name: dask-worker
args: [dask-worker, $(DASK_SCHEDULER_ADDRESS), --nthreads, '2', --memory-limit, 8GB]
restartPolicy: Never
Examples
--------
>>> cluster = KubeCluster.from_yaml('pod.yaml', namespace='my-ns') # doctest: +SKIP
See Also
--------
KubeCluster.from_dict
"""
if not yaml:
raise ImportError(
"PyYaml is required to use yaml functionality, please install it!"
)
with open(yaml_path) as f:
d = yaml.safe_load(f)
d = dask.config.expand_environment_variables(d)
return cls.from_dict(d, **kwargs)
@property
def namespace(self):
return self.pod_template.metadata.namespace
@property
def name(self):
return self.pod_template.metadata.generate_name
def scale(self, n):
# A shim to maintain backward compatibility
# https://github.com/dask/distributed/issues/3054
maximum = dask.config.get("kubernetes.count.max")
if maximum is not None and maximum < n:
logger.info(
"Tried to scale beyond maximum number of workers %d > %d", n, maximum
)
n = maximum
return super().scale(n)
async def _logs(self, scheduler=True, workers=True):
""" Return logs for the scheduler and workers
Parameters
----------
scheduler : boolean
Whether or not to collect logs for the scheduler
workers : boolean or Iterable[str], optional
A list of worker addresses to select.
Defaults to all workers if `True` or no workers if `False`
Returns
-------
logs: Dict[str]
A dictionary of logs, with one item for the scheduler and one for
each worker
"""
logs = Logs()
if scheduler:
logs["Scheduler"] = await self.scheduler.logs()
if workers:
worker_logs = await asyncio.gather(
*[w.logs() for w in self.workers.values()]
)
for key, log in zip(self.workers, worker_logs):
logs[key] = log
return logs
def _cleanup_resources(namespace, labels):
""" Remove all pods with these labels in this namespace """
import kubernetes
core_api = kubernetes.client.CoreV1Api()
pods = core_api.list_namespaced_pod(namespace, label_selector=format_labels(labels))
for pod in pods.items:
try:
core_api.delete_namespaced_pod(pod.metadata.name, namespace)
logger.info("Deleted pod: %s", pod.metadata.name)
except kubernetes.client.rest.ApiException as e:
# ignore error if pod is already removed
if e.status != 404:
raise
services = core_api.list_namespaced_service(
namespace, label_selector=format_labels(labels)
)
for service in services.items:
try:
core_api.delete_namespaced_service(service.metadata.name, namespace)
logger.info("Deleted service: %s", service.metadata.name)
except kubernetes.client.rest.ApiException as e:
# ignore error if service is already removed
if e.status != 404:
raise
def format_labels(labels):
""" Convert a dictionary of labels into a comma separated string """
if labels:
return ",".join(["{}={}".format(k, v) for k, v in labels.items()])
else:
return ""
def _namespace_default():
"""
Get current namespace if running in a k8s cluster
If not in a k8s cluster with service accounts enabled, default to
'default'
Taken from https://github.com/jupyterhub/kubespawner/blob/master/kubespawner/spawner.py#L125
"""
ns_path = "/var/run/secrets/kubernetes.io/serviceaccount/namespace"
if os.path.exists(ns_path):
with open(ns_path) as f:
return f.read().strip()
return "default"
def escape(s):
valid_characters = string.ascii_letters + string.digits + "-"
return "".join(c for c in s if c in valid_characters)