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

   ### Apache Airflow version
   
   2.7.1
   
   ### What happened
   
   If trying to wait for a DAG currently in a `deferred` state using the 
ExternalTaskSensor in deferrable mode, the sensor doesn't consider that the DAG 
is running and fails after 60 seconds.
   
   After looking at the code, I believe it is a matter of adding `deferred` to 
the method `count_running_dags`:
   
   ```
    dags = (
                  session.query(func.count("*"))
                  .filter(
                           TaskInstance.dag_id == self.dag_id,
                           
TaskInstance.execution_date.in_(self.execution_dates),
                           TaskInstance.state.in_([TaskInstanceState.RUNNING, 
TaskInstanceState.SUCCESS, TaskInstanceState.DEFERRED]),
                 )
                 .scalar()
   )
   ```
   
   I tested this code on my local machine and it worked as expected when 
waiting for a deferred DAG.
   
   Maybe @bkossakowska could check this out?
   
   ### What you think should happen instead
   
   _No response_
   
   ### How to reproduce
   
   1. Create `dag1` that contains a simple task that defers itself (e.g. 
TimeSensorAsync)
   2. Create `dag2` which contains an ExternalTaskSensor running in deferrable 
mode
   3. Run at the same time and see how `dag2`'s task fails with an 
`airflow.exceptions.AirflowException: Dag was not started within 1 minute, 
assuming fail.`
   
   ### Operating System
   
   Ubuntu 22.04.3 LTS
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Virtualenv installation
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   Similarly to #34204 I'd love to submit a PR, but have no available time to 
properly do so or the knowledge to make sure the proposed solution actually 
works in all cases.
   
   ### 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to