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 2022/04/06 14:29:06 UTC

[GitHub] [airflow] Gollum999 opened a new issue, #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

Gollum999 opened a new issue, #22782:
URL: https://github.com/apache/airflow/issues/22782

   ### Apache Airflow version
   
   2.2.4
   
   ### What happened
   
   When using `ExternalTaskSensor`, if a Jinja template is used in `external_task_id` or `external_task_ids`, that template will not be expanded, causing the sensor to always fail.
   
   ### What you think should happen instead
   
   Ideally the template should be expanded.  If we can't make that work for whatever reason, we should remove `external_task_id` from the list of valid template fields.
   
   ### How to reproduce
   
   ```
   #!/usr/bin/env python3
   from datetime import datetime
   
   from airflow import DAG
   from airflow.operators.dummy import DummyOperator
   from airflow.sensors.external_task import ExternalTaskSensor
   
   
   with DAG('dag1', start_date=datetime(2022, 4, 1), schedule_interval='@daily', is_paused_upon_creation=False) as dag1:
       DummyOperator(task_id='task_123')
   
   
   with DAG('dag2', start_date=datetime(2022, 4, 1), schedule_interval='@daily', is_paused_upon_creation=False) as dag2:
       ExternalTaskSensor(
           task_id='not_using_params',
           external_dag_id='dag1',
           external_task_id='task_123',
           check_existence=True,
       )
       ExternalTaskSensor(
           task_id='using_external_task_id',
           external_dag_id='{{ params.dag_name }}',
           external_task_id='{{ params.task_name }}',
           check_existence=True,
           params={
               'dag_name': 'dag1',
               'task_name': 'task_123',
           },
       )
       ExternalTaskSensor(
           task_id='using_external_task_ids',
           external_dag_id='{{ params.dag_name }}',
           external_task_ids=['{{ params.task_name }}'],
           check_existence=True,
           params={
               'dag_name': 'dag1',
               'task_name': 'task_123',
           },
       )
   ```
   
   Here are some relevant snippets from the task logs:
   
   'not_using_params':
   ```
   [2022-04-06, 04:25:40 CDT] {external_task.py:169} INFO - Poking for tasks ['task_123'] in dag dag1 on 2022-04-01T00:00:00+00:00 ... 
   ```
   
   'using_external_task_id':
   ```
   [2022-04-06, 04:25:41 CDT] {external_task.py:169} INFO - Poking for tasks ['{{ params.task_name }}'] in dag dag1 on 2022-04-01T00:00:00+00:00 ... 
   ```
   
   'using_external_task_ids':
   ```
   [2022-04-06, 04:25:43 CDT] {external_task.py:169} INFO - Poking for tasks ['{{ params.task_name }}'] in dag dag1 on 2022-04-01T00:00:00+00:00 ... 
   ```
   
   As we can see, the templated versions correctly expand the `dag_name` parameter, but not `task_name`.
   
   ### Operating System
   
   CentOS 7.4
   
   ### Versions of Apache Airflow Providers
   
   N/A
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   Standalone
   
   ### Anything else
   
   Maybe a separate issue, but worth noting: `ExternalTaskSensor` does not even list `external_task_ids` as a valid template field, though it seems like it should.  In the above example, the "Rendered Template" works correctly for `'using_external_task_id'`, but not for `'using_external_task_ids'`.
   
   ### Are you willing to submit PR?
   
   - [ ] 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: commits-unsubscribe@airflow.apache.org.apache.org

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


[GitHub] [airflow] potiuk closed issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)
URL: https://github.com/apache/airflow/issues/22782


-- 
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: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

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

   Would you lie to add external_task_ids to templated fields? @Gollum999 ? Seems like easy contribution.


-- 
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: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] tirkarthi commented on issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

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

   Templating has been added in https://issues.apache.org/jira/browse/AIRFLOW-1480


-- 
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: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] tirkarthi commented on issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

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

   Even in `using_external_task_id` example the value is not rendered. `self.external_task_ids` is always logged. When `external_task_id` is passed in the constructor then `external_task_ids` is set as a list of the value. In this case the unrendered template value is set and logged in poke method. Regarding `external_task_ids` being made as a template field there are some validations done for `external_task_ids` in constructor where templates might not be useful and values are rendered during task runs.
   
   https://github.com/apache/airflow/blob/0592bfd85631ed3109d68c8ec9aa57f0465d90b3/airflow/sensors/external_task.py#L119-L120
   
   https://github.com/apache/airflow/blob/0592bfd85631ed3109d68c8ec9aa57f0465d90b3/airflow/sensors/external_task.py#L162-L167
   
   https://github.com/apache/airflow/blob/0592bfd85631ed3109d68c8ec9aa57f0465d90b3/airflow/sensors/external_task.py#L122-L129
   
   Test case for issue. It will hang without any output.
   
   ```python
       def test_external_task_sensor_params(self):                                                                                                                              
           self.test_time_sensor()                                                                                                                                              
           op = ExternalTaskSensor(                                                                                                                                             
               task_id='test_external_task_sensor_check',                                                                                                                       
               external_dag_id='{{ params.dag_id }}',                                                                                                                           
               external_task_id='{{ params.task_id }}',                                                                                                                         
               params={                                                                                                                                                         
                   'dag_id': TEST_DAG_ID,                                                                                                                                       
                   'task_id': TEST_TASK_ID,                                                                                                                                     
               },                                                                                                                                                               
               dag=self.dag,                                                                                                                                                    
           )                                                                                                                                                                    
                                                                                                                                                                                
           with self.assertLogs(op.log, level=logging.INFO) as cm:                                                                                                              
               op.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)                                                                                 
               assert (                                                                                                                                                         
                   f"INFO:airflow.task.operators:Poking for tasks ['{TEST_TASK_ID}'] in dag unit_test_dag on {DEFAULT_DATE.isoformat()} ... " in cm.output                      
               ) 
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] tirkarthi commented on issue #22782: ExternalTaskSensor does not properly expand templates in external_task_id(s)

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

   @potiuk Thanks for your suggestion to support templates for  `external_task_ids`. I have updated my PR along with tests in https://github.com/apache/airflow/pull/22809


-- 
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: commits-unsubscribe@airflow.apache.org

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