You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/01/26 20:27:21 UTC

[GitHub] [airflow] armandleopold opened a new issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

armandleopold opened a new issue #13918:
URL: https://github.com/apache/airflow/issues/13918


   **Apache Airflow version**: 2.0.0
   
   **Kubernetes version (if you are using kubernetes)** 1.15.15
   
   **What happened**:
   
   If you use the **KubernetesPodOperator** with **LocalExecutor** and you use a **pod_template_file**, the pod created doesn't have metadata like : 
   - dag_id
   - task_id
   - ...
   
   I want to have a ``privileged_escalation=True`` pod, launched by a KubernetesPodOperator but without the KubernetesExecutor.
   Is it possible ?
   
   **What you expected to happen**:
   
   Have the pod launched with privileged escalation & metadata & correct pod-name override.
   
   **How to reproduce it**:
   
   * have a pod template file : 
   
   **privileged_runner.yaml** :  
   ```yaml
   apiVersion: v1
   kind: Pod
   metadata:
     name: privileged-pod
   spec:
     containers:
     - name: base
       securityContext:
         allowPrivilegeEscalation: true
         privileged: true
   ```
   
   * have a DAG file with KubernetesOperator in it : 
   
   **my-dag.py** : 
   ```yaml
   ##=========================================================================================##
   ##                                      CONFIGURATION 
   
   from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
   from airflow.operators.dummy_operator import DummyOperator
   from airflow.kubernetes.secret import Secret
   from kubernetes.client import models as k8s
   from airflow.models import Variable
   from datetime import datetime, timedelta
   from airflow import DAG
   
   env = Variable.get("process_env")
   namespace = Variable.get("namespace")
   
   default_args = {
       'owner': 'airflow',
       'depends_on_past': False,
       'email_on_failure': False,
       'email_on_retry': False,
       'retries': 1,
       'retry_delay': timedelta(minutes=5)
   }
   
   ##==============================##
   
   ## Définition du DAG
   dag = DAG(
       'transfert-files-to-nexus',
       start_date=datetime.utcnow(),
       schedule_interval="0 2 * * *",
       default_args=default_args,
       max_active_runs=1
   )
   ##=========================================================================================##
   
   ## Définition des tâches
   start = DummyOperator(task_id='start', dag=dag)
   end = DummyOperator(task_id='end', dag=dag)
   
   transfertfile = KubernetesPodOperator(namespace=namespace,
                                task_id="transfertfile",
                                name="transfertfile",
                                image="registrygitlab.fr/docker-images/python-runner:1.8.22",
                                image_pull_secrets="registrygitlab-curie",
                                pod_template_file="/opt/bitnami/airflow/dags/git-airflow-dags/privileged_runner.yaml",
                                is_delete_operator_pod=False,
                                get_logs=True,
                                dag=dag)
                                
   ## Enchainement des tâches
   start >> transfertfile >> end
   ```
   
   **Anything else we need to know**:
   I know that we have to use the ``KubernetesExecutor`` in order to have the **metadata**, but even if you use the ``KubernetesExecutor``, the fact that you have to use the **pod_template_file** for the ``KubernetesPodOperator`` makes no change, because in either ``LocalExecutor`` / ``KubernetesExecutor``you will endup with no pod name override correct & metadata.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-769828478


   @lgov :  Interessting, i will investigate this. I will keep you in touch 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold edited a comment on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold edited a comment on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-780559692


   Hello,
   - I have had chmod -R 777 on the git-repo-syncer. 
   - The airflow-scheduler is root and the dags files have 777 permissions.
   - The path `/opt/bitnami/airflow/dags/git-airflow-dags/privileged_runner.yaml` is correct in the airflow-scheduler pod
   
   It still runs my pod with the hardcoded metadata name : privileged-pod.
   
   Do someone have any idea ?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-825141550


   A combination of https://github.com/apache/airflow/pull/14186 and https://github.com/apache/airflow/pull/15492 should fix this issue


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aviau commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
aviau commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-783702733


   Same issue here and I can confirm that the template is read because it sucessfully launches with the provided config.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil closed issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
kaxil closed issue #13918:
URL: https://github.com/apache/airflow/issues/13918


   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] lgov commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
lgov commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-769122329


   This might not be the issue you are having, but I noticed that if the pod template file that you're using in your KubernetesPodOperator can not be read (e.g. because the volume is not mounted in the path you specified), then K.P.O will try to launch a pod with a template that has all the right keys but empty values.
   
   In that specific scenario, you'll not get an error messages like 'pod_template_file not found', but you'll get an error from Kubernetes like 'namespace value not filled in' or something similar.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-785242699


   Cool ! Will it be released soon ? 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-780559692


   Hello, i have had chmod -R 777 on the git-repo-syncer. 
   The airflow-scheduler is root and the files have 777 permissions.
   It still runs my pod with the hardcoded metadata name : privileged-pod.
   
   Do someone have any idea ?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-813521222


   @armandleopold Can you try with Kubernetes Provider 1.0.2 please:
   
   https://pypi.org/project/apache-airflow-providers-cncf-kubernetes/1.0.2/
   
   ```
   pip install apache-airflow-providers-cncf-kubernetes==1.0.2
   ```
   
   and see if it fixes for you. (#14186 was included in that release)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-822387474


   @kaxil Haven't got the time to do so, a little busy at the moment. Not sure if i can try things for now.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold edited a comment on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold edited a comment on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-780559692


   Hello,
   - I have had chmod -R 777 on the git-repo-syncer. 
   - The airflow-scheduler is root and the files have 777 permissions.
   - The path `/opt/bitnami/airflow/dags/git-airflow-dags/privileged_runner.yaml` is correct in the airflow-scheduler pod
   
   It still runs my pod with the hardcoded metadata name : privileged-pod.
   
   Do someone have any idea ?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Dr-Denzy commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
Dr-Denzy commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-811171522


   I am on it.
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dimberman commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
dimberman commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-779392137


   @armandleopold any update on this?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-810629049


   cc @Dr-Denzy 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] Dr-Denzy commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
Dr-Denzy commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-821305671


   Using airflow 2.0.0 ... kubectl description of the `privileged-pod` gotten did not show task metadata (dag_id, task_id, etc).
   However, `airflow 2.0.1` with `apache-airflow-providers-cncf-kubernetes==1.0.2` yielded the right pod name but still did not show the task metadata.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] aviau commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
aviau commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-784566801


   I deployed https://github.com/apache/airflow/pull/14186 and it fixes the issue for the pod name.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] armandleopold commented on issue #13918: KubernetesPodOperator with pod_template_file = No Metadata & Wrong Pod Name

Posted by GitBox <gi...@apache.org>.
armandleopold commented on issue #13918:
URL: https://github.com/apache/airflow/issues/13918#issuecomment-805746840


   Found a wordaround using **`full_pod_spec`**
   
   ```yaml
   
   full_pod_spec=k8s.V1Pod(
   metadata=k8s.V1ObjectMeta(
   name="hardcodedname"),
   spec=k8s.V1PodSpec(
   containers=[k8s.V1Container(
   name="base",
   security_context=k8s.V1SecurityContext(
   allow_privilege_escalation=True,privileged=True))])),
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org