24
24
import copy
25
25
import hashlib
26
26
import re
27
+ try :
28
+ from inspect import signature
29
+ except ImportError :
30
+ # Python 2.7
31
+ from funcsigs import signature # type: ignore
32
+ import os
27
33
import uuid
34
+ from functools import reduce
28
35
29
36
import kubernetes .client .models as k8s
37
+ import yaml
38
+ from kubernetes .client .api_client import ApiClient
30
39
40
+ from airflow .exceptions import AirflowConfigException
31
41
from airflow .version import version as airflow_version
32
42
33
43
MAX_LABEL_LEN = 63
34
44
35
45
MAX_POD_ID_LEN = 253
36
46
37
47
38
- class PodDefaults :
48
+ class PodDefaults ( object ) :
39
49
"""
40
50
Static defaults for the PodGenerator
41
51
"""
52
+
53
+ def __init__ (self ):
54
+ pass
55
+
42
56
XCOM_MOUNT_PATH = '/airflow/xcom'
43
57
SIDECAR_CONTAINER_NAME = 'airflow-xcom-sidecar'
44
58
XCOM_CMD = 'trap "exit 0" INT; while true; do sleep 30; done;'
@@ -82,7 +96,7 @@ def make_safe_label_value(string):
82
96
return safe_label
83
97
84
98
85
- class PodGenerator :
99
+ class PodGenerator ( object ) :
86
100
"""
87
101
Contains Kubernetes Airflow Worker configuration logic
88
102
@@ -147,9 +161,11 @@ class PodGenerator:
147
161
:param dnspolicy: Specify a dnspolicy for the pod
148
162
:type dnspolicy: str
149
163
:param schedulername: Specify a schedulername for the pod
150
- :type schedulername: str
151
- :param pod: The fully specified pod.
152
- :type pod: kubernetes.client.models.V1Pod
164
+ :type schedulername: Optional[str]
165
+ :param pod: The fully specified pod. Mutually exclusive with `path_or_string`
166
+ :type pod: Optional[kubernetes.client.models.V1Pod]
167
+ :param pod_template_file: Path to YAML file. Mutually exclusive with `pod`
168
+ :type pod_template_file: Optional[str]
153
169
:param extract_xcom: Whether to bring up a container for xcom
154
170
:type extract_xcom: bool
155
171
"""
@@ -167,8 +183,8 @@ def __init__(
167
183
node_selectors = None ,
168
184
ports = None ,
169
185
volumes = None ,
170
- image_pull_policy = 'IfNotPresent' ,
171
- restart_policy = 'Never' ,
186
+ image_pull_policy = None ,
187
+ restart_policy = None ,
172
188
image_pull_secrets = None ,
173
189
init_containers = None ,
174
190
service_account_name = None ,
@@ -183,9 +199,16 @@ def __init__(
183
199
schedulername = None ,
184
200
priority_class_name = None ,
185
201
pod = None ,
202
+ pod_template_file = None ,
186
203
extract_xcom = False ,
187
204
):
188
- self .ud_pod = pod
205
+ self .validate_pod_generator_args (locals ())
206
+
207
+ if pod_template_file :
208
+ self .ud_pod = self .deserialize_model_file (pod_template_file )
209
+ else :
210
+ self .ud_pod = pod
211
+
189
212
self .pod = k8s .V1Pod ()
190
213
self .pod .api_version = 'v1'
191
214
self .pod .kind = 'Pod'
@@ -348,37 +371,7 @@ def from_obj(obj):
348
371
'iam.cloud.google.com/service-account' : gcp_service_account_key
349
372
})
350
373
351
- pod_spec_generator = PodGenerator (
352
- image = namespaced .get ('image' ),
353
- envs = namespaced .get ('env' ),
354
- cmds = namespaced .get ('cmds' ),
355
- args = namespaced .get ('args' ),
356
- labels = namespaced .get ('labels' ),
357
- node_selectors = namespaced .get ('node_selectors' ),
358
- name = namespaced .get ('name' ),
359
- ports = namespaced .get ('ports' ),
360
- volumes = namespaced .get ('volumes' ),
361
- volume_mounts = namespaced .get ('volume_mounts' ),
362
- namespace = namespaced .get ('namespace' ),
363
- image_pull_policy = namespaced .get ('image_pull_policy' ),
364
- restart_policy = namespaced .get ('restart_policy' ),
365
- image_pull_secrets = namespaced .get ('image_pull_secrets' ),
366
- init_containers = namespaced .get ('init_containers' ),
367
- service_account_name = namespaced .get ('service_account_name' ),
368
- resources = resources ,
369
- annotations = namespaced .get ('annotations' ),
370
- affinity = namespaced .get ('affinity' ),
371
- hostnetwork = namespaced .get ('hostnetwork' ),
372
- tolerations = namespaced .get ('tolerations' ),
373
- security_context = namespaced .get ('security_context' ),
374
- configmaps = namespaced .get ('configmaps' ),
375
- dnspolicy = namespaced .get ('dnspolicy' ),
376
- schedulername = namespaced .get ('schedulername' ),
377
- pod = namespaced .get ('pod' ),
378
- extract_xcom = namespaced .get ('extract_xcom' ),
379
- )
380
-
381
- return pod_spec_generator .gen_pod ()
374
+ return PodGenerator (** namespaced ).gen_pod ()
382
375
383
376
@staticmethod
384
377
def reconcile_pods (base_pod , client_pod ):
@@ -495,12 +488,73 @@ def construct_pod(
495
488
name = pod_id
496
489
).gen_pod ()
497
490
498
- # Reconcile the pod generated by the Operator and the Pod
499
- # generated by the .cfg file
500
- pod_with_executor_config = PodGenerator .reconcile_pods (worker_config ,
501
- kube_executor_config )
502
- # Reconcile that pod with the dynamic fields.
503
- return PodGenerator .reconcile_pods (pod_with_executor_config , dynamic_pod )
491
+ # Reconcile the pods starting with the first chronologically,
492
+ # Pod from the airflow.cfg -> Pod from executor_config arg -> Pod from the K8s executor
493
+ pod_list = [worker_config , kube_executor_config , dynamic_pod ]
494
+
495
+ return reduce (PodGenerator .reconcile_pods , pod_list )
496
+
497
+ @staticmethod
498
+ def deserialize_model_file (path ):
499
+ """
500
+ :param path: Path to the file
501
+ :return: a kubernetes.client.models.V1Pod
502
+
503
+ Unfortunately we need access to the private method
504
+ ``_ApiClient__deserialize_model`` from the kubernetes client.
505
+ This issue is tracked here; https://github.com/kubernetes-client/python/issues/977.
506
+ """
507
+ api_client = ApiClient ()
508
+ if os .path .exists (path ):
509
+ with open (path ) as stream :
510
+ pod = yaml .safe_load (stream )
511
+ else :
512
+ pod = yaml .safe_load (path )
513
+
514
+ # pylint: disable=protected-access
515
+ return api_client ._ApiClient__deserialize_model (pod , k8s .V1Pod )
516
+
517
+ @staticmethod
518
+ def validate_pod_generator_args (given_args ):
519
+ """
520
+ :param given_args: The arguments passed to the PodGenerator constructor.
521
+ :type given_args: dict
522
+ :return: None
523
+
524
+ Validate that if `pod` or `pod_template_file` are set that the user is not attempting
525
+ to configure the pod with the other arguments.
526
+ """
527
+ pod_args = list (signature (PodGenerator ).parameters .items ())
528
+
529
+ def predicate (k , v ):
530
+ """
531
+ :param k: an arg to PodGenerator
532
+ :type k: string
533
+ :param v: the parameter of the given arg
534
+ :type v: inspect.Parameter
535
+ :return: bool
536
+
537
+ returns True if the PodGenerator argument has no default arguments
538
+ or the default argument is None, and it is not one of the listed field
539
+ in `non_empty_fields`.
540
+ """
541
+ non_empty_fields = {
542
+ 'pod' , 'pod_template_file' , 'extract_xcom' , 'service_account_name' , 'image_pull_policy' ,
543
+ 'restart_policy'
544
+ }
545
+
546
+ return (v .default is None or v .default is v .empty ) and k not in non_empty_fields
547
+
548
+ args_without_defaults = {k : given_args [k ] for k , v in pod_args if predicate (k , v ) and given_args [k ]}
549
+
550
+ if given_args ['pod' ] and given_args ['pod_template_file' ]:
551
+ raise AirflowConfigException ("Cannot pass both `pod` and `pod_template_file` arguments" )
552
+ if args_without_defaults and (given_args ['pod' ] or given_args ['pod_template_file' ]):
553
+ raise AirflowConfigException (
554
+ "Cannot configure pod and pass either `pod` or `pod_template_file`. Fields {} passed." .format (
555
+ list (args_without_defaults .keys ())
556
+ )
557
+ )
504
558
505
559
506
560
def merge_objects (base_obj , client_obj ):
0 commit comments