From ae9a5ed412bf010775fe9beb8f09cd3bba8a0efe Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Wed, 14 Jul 2021 17:33:24 +0100 Subject: [PATCH] [backport] Fix bug with `executor_config` and Volumes Without this fix the following DAG will not create Volumes & Volume Mounts and fais: ```python dag1 = DAG( dag_id='example_k8s-dags', schedule_interval='@once', start_date=datetime(2020, 1, 1) ) p = PythonOperator( task_id='test', dag=dag1, python_callable=lambda: 1, executor_config={ "KubernetesExecutor": { 'resources': { 'limits': {'memory': '200Mi', 'cpu': '100m'}, 'requests': {'memory': '100Mi', 'cpu': '100m'} }, "volumes": [ { "name": "tmp-volume", "hostPath": {"path": "/tmp/"}, }, ], "volume_mounts": [ { "mountPath": "/tmp/", "name": "tmp-volume", }, ], } } ) ``` (cherry picked from commit 7813076ac76a0bea858658cd30b224341c637a0b) (cherry picked from commit e268afd5c943453d62f2cc3df3610eb87557b06e) --- airflow/kubernetes/pod_generator.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/kubernetes/pod_generator.py b/airflow/kubernetes/pod_generator.py index 83597d278357b..77db214c8fccb 100644 --- a/airflow/kubernetes/pod_generator.py +++ b/airflow/kubernetes/pod_generator.py @@ -36,7 +36,7 @@ import kubernetes.client.models as k8s import yaml from kubernetes.client.api_client import ApiClient -from airflow.contrib.kubernetes.pod import _extract_volume_mounts +from airflow.contrib.kubernetes.pod import _extract_volume_mounts, _extract_volumes from airflow.exceptions import AirflowConfigException from airflow.version import version as airflow_version @@ -262,7 +262,7 @@ def __init__( self.spec.affinity = affinity self.spec.service_account_name = service_account_name self.spec.init_containers = init_containers - self.spec.volumes = volumes or [] + self.spec.volumes = [v.to_k8s_client_obj() for v in _extract_volumes(volumes)] or [] self.spec.node_selector = node_selectors self.spec.restart_policy = restart_policy self.spec.priority_class_name = priority_class_name