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

   ### Apache Airflow version
   
   2.7.1
   
   ### What happened
   
   If a DAG (`dag1`) is running and another DAG (`dag2`) has an 
ExternalTaskSensor (`task-externalsensor`) that checks a task on `dag1`,  
`task-externalsensor` will fail unless `dag1`'s task finishes in under 60 
seconds.
   
   After checking the code, I believe the `while` block in the 
[trigger](https://github.com/apache/airflow/blob/main/airflow/triggers/external_task.py#L90)
 should be split in two, as follows:
   
   ```
   # Wait for the dag to start execution
   # Maybe the timeout should be configurable?
   while True:
       try:
           delta = utcnow() - self.trigger_start_time
           if delta.total_seconds() < self._timeout_sec:
               # mypy confuses typing here
               if await self.count_running_dags() == 0:  # type: 
ignore[call-arg]
                   self.log.info("Waiting for DAG to start execution...")
                   await asyncio.sleep(self.poll_interval)
               else:
                   break
           else:
               yield TriggerEvent({"status": "timeout"})
               return
       except Exception:
           yield TriggerEvent({"status": "failed"})
           return
   
   # Wait for the external task to finish
   # Maybe out a timeout here?
   while True:
       try:
           if await self.count_tasks() == len(self.execution_dates):  # type: 
ignore[call-arg]
               yield TriggerEvent({"status": "success"})
               return
           self.log.info("Task is still running, sleeping for %s seconds...", 
self.poll_interval)
           await asyncio.sleep(self.poll_interval)
       except Exception:
           yield TriggerEvent({"status": "failed"})
           return
   ```
   
   I tested this code on my local machine and it worked as expected with `dag1` 
running for 120 seconds. 
   
   Maybe @bkossakowska could check this out?
   
   ### What you think should happen instead
   
   _No response_
   
   ### How to reproduce
   
   1. Create `dag1` which contains a simple task (sleep for > 60 seconds)
   2. Create `dag2` which contains an ExternalTaskSensor checking for `dag1`'s 
task and runs in deferrable mode
   3. Execute them at the same time. As soon as `dag2`'s trigger has spent 60 
seconds, it will fail, even if `dag1`'s task is still running.
   
   ### Operating System
   
   Ubuntu 22.04.3 LTS
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Virtualenv installation
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   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