brouberol opened a new issue, #46373:
URL: https://github.com/apache/airflow/issues/46373
### Apache Airflow Provider(s)
cncf-kubernetes
### Versions of Apache Airflow Providers
apache-airflow-providers-amazon==9.2.0
apache-airflow-providers-apache-hdfs==4.7.0
apache-airflow-providers-apache-hive==9.0.0
apache-airflow-providers-apache-spark==4.8.1
apache-airflow-providers-cncf-kubernetes==10.1.0
apache-airflow-providers-common-compat==1.3.0
apache-airflow-providers-common-io==1.5.0
apache-airflow-providers-common-sql==1.21.0
apache-airflow-providers-fab==1.5.2
apache-airflow-providers-ftp==3.12.0
apache-airflow-providers-http==5.0.0
apache-airflow-providers-imap==3.8.0
apache-airflow-providers-postgres==6.0.0
apache-airflow-providers-smtp==1.9.0
apache-airflow-providers-sqlite==4.0.0
### Apache Airflow version
2.10.3
### Operating System
Debian 11 Bullseye
### Deployment
Other 3rd-party Helm chart
### Deployment details
Our airflow instance is running in Kubernetes, and uses the
`KubernetesExecutor` to run tasks as Kubernetes pods. It uses the `inCluster`
config setup, to get permissions from its serviceaccount token.
### What happened
We deploy task as Kubernetes pods, using a `pod_template_file` configured in
`airflow.cfg`. However, some tasks make use the of the `KubernetesPodOperator`
to themselves create a pod to run an ad-hoc command (that might or might not be
python code). We have defined a second pod template file that contains extra
configuration.
```diff
~ ❯ diff -u default_pod_template.yaml
kubernetes_executor_pod_template_kubeapi_enabled.yaml
--- default_pod_template.yaml 2025-02-03 12:02:46
+++ kubernetes_executor_pod_template_kubeapi_enabled.yaml 2025-02-03
12:03:47
@@ -7,6 +7,7 @@
release: production
routed_via: production
component: task-pod
+ kubeapi_enabled: 'True'
spec:
restartPolicy: Never
hostAliases:
@@ -16,6 +17,7 @@
- ip: 10.64.36.112
hostnames:
- an-test-master1002.eqiad.wmnet
+ serviceAccountName: airflow
volumes:
- configMap:
```
The label is used to allow egress to the Kubernetes API server, via a Calico
Networkpolicy, and the `serviceAccount` is used to make sure that the task pod
has the required RBAC to create and delete the pods via the
`KubernetesPodOperator`.
We pass that non-default `pod_template_file` to the executor via
`executor_options['pod_template_file']`
([link](https://gitlab.wikimedia.org/repos/data-engineering/airflow-dags/-/blob/96a54a46653073ddc1b5c6d95ce3eb01146d5c47/wmf_airflow_common/operators/kubernetes.py#L16))
I realized that while the label and serviceAccount appeared in the output of
`kubectl get pod <pod-name> -oyaml`, they did _not_ appear in the `K8s Pod
Spec` pane in the Airflow UI. See this example:
<img width="868" alt="Image"
src="https://github.com/user-attachments/assets/9c9a4b20-b68f-4fd1-a187-8043847bb405"
/>
_The `kubeapi_enabled` label is missing_
I had a look at the rendered pod spec in the database (in the
`rendered_task_instance_fields` table) and both the label and the service
account were missing from there as well, which indicates that the code
responsible for inserting that spec in the database in the first place is at
fault.
### What you think should happen instead
I tracked down the code responsible for the rendering of the task instance
k8s pod spec to
[`render_k8s_pod_yaml`](https://github.com/apache/airflow/blob/main/providers/src/airflow/providers/cncf/kubernetes/template_rendering.py#L35),
which includes the following line
```python
base_worker_pod=PodGenerator.deserialize_model_file(kube_config.pod_template_file),
```
itself defined
[here](https://github.com/apache/airflow/blob/f871e015ce97423838fc17faca68ff1fc1fbed17/providers/src/airflow/providers/cncf/kubernetes/kube_config.py#L37)
as
```python
self.pod_template_file = conf.get(self.kubernetes_section,
"pod_template_file", fallback=None)
```
Nowhere in that function are we looking at a potential `pod_template_file`
override passed via the `executor_options`.
### How to reproduce
To reproduce this issue, you need to run airflow with `KubernetesExecutor`,
set `kubernetes_executor.pod_template_file` as the path of an existing yaml pod
template, and also provide another pod template (referenced later as
`/path/to/custom_pod_template_file.yaml`). This custom template should differ
from the default one in some way (added or removed fields, for example).
Then run the following DAG, and inspect its pod spec:
```python
from datetime import datetime
from airflow.providers.cncf.kubernetes.operators.pod import
KubernetesPodOperator
from airflow import DAG
with DAG(
...
) as dag:
run_k8s_pod = KubernetesPodOperator(
task_id="run-cat-os-release",
name="run-cat-os-release",
# the container can run anything, not just python code
image="debian:bookworm",
cmds=["/usr/bin/cat"],
arguments=["/etc/os-release"],
executor_config={"pod_template_file":
"/path/to/custom_pod_template_file.yaml"}
)
run_k8s_pod
```
### Anything else
_No response_
### Are you willing to submit PR?
- [x] Yes I am willing to submit a PR!
### Code of Conduct
- [x] I agree to follow this project's [Code of
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]