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

   ### Apache Airflow version
   
   main (development)
   
   ### If "Other Airflow 2 version" selected, which one?
   
   _No response_
   
   ### What happened?
   
   When there is a manual dagrun, it is created with the logical date as data 
interval end. In case of scheduled runs the logical date is the data interval 
start. In this case when a future run is scheduled there is a check for 
existing dagruns in the loop with respect to execution date (logical date) 
which causes the dagrun to be skipped since it exists from the manual run with 
a different data interval.
   
   This looks like a known issue in 2.x and can to be changed in 3.x . Related 
issue https://github.com/apache/airflow/issues/41014
   
   
https://github.com/apache/airflow/blob/4d849e87ecce654e100c36dca3d0fbe19bcd8b6b/airflow/jobs/scheduler_job_runner.py#L1314-L1348
   
   ### What you think should happen instead?
   
   Scheduled dag run should be executed though a manual run with same execution 
date exists.
   
   ### How to reproduce
   
   1. Place the below dag and parse it.
   2. Before unpausing the dag trigger a manual run with logical date as 
"2024-07-01T00:00:00+00".
   3. A manual run is triggered with run_id as 
manual__2024-07-01T00:00:00+00:00 and exeuction_date as 2024-07-01 
00:00:00+00:00. Data interval is (2024-06-01, 00:00:00 UTC, 2024-07-01, 
00:00:00 UTC)
   4. The run with data interval (2024-07-01T00:00:00+00:00, 
2024-08-01T00:00:00+00:00) is not scheduled.
   
   ```python
   from datetime import datetime, timedelta
   
   from airflow import DAG
   from airflow.decorators import task
   
   
   with DAG(
       dag_id="execution_date_debug",
       start_date=datetime(2024, 1, 1),
       catchup=True,
       schedule_interval="0 0 1 * *",
   ) as dag:
   
       @task
       def empty():
           pass
   
       empty()
   ```
   
   ### Operating System
   
   Ubuntu
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   _No response_
   
   ### Anything else?
   
   _No response_
   
   ### 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