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

   ### Apache Airflow version
   
   2.3.4
   
   ### What happened
   
   When the dagrun already exists and is backfilled with the flags 
`--reset-dagruns --yes`, the dag run will not have a start date. This is 
because reset_dagruns calls 
[clear_task_instances](https://github.com/apache/airflow/blob/main/airflow/models/dag.py#L2020)
 which [sets the dagrun start date to 
None](https://github.com/apache/airflow/blob/main/airflow/models/taskinstance.py#L286-L291).
   
   Since the dagrun goes into running via BackfillJob rather than the 
SchedulerJob, the start date is not set. This doesn't happen to a new dagrun 
created by a BackfillJob because the [start date is determined at 
creation](https://github.com/apache/airflow/blob/main/airflow/jobs/backfill_job.py#L310-L320).
   
   Here is a recreation of the behaviour.
   
   First run of the backfill dagrun. No odd warnings and start date exists for 
Airflow to calculate the duration.
   
   ```
   astro@75512ab5e882:/usr/local/airflow$ airflow dags backfill -s 2021-12-01 
-e 2021-12-01 test_module
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528: 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
     option = self._get_environment_variables(deprecated_key, 
deprecated_section, key, section)
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
   
/usr/local/lib/python3.9/site-packages/airflow/cli/commands/dag_command.py:57 
PendingDeprecationWarning: --ignore-first-depends-on-past is deprecated as the 
value is always set to True
   [2022-08-25 21:29:55,574] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dags
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
   Nothing to clear.
   [2022-08-25 21:29:55,650] {executor_loader.py:105} INFO - Loaded executor: 
LocalExecutor
   [2022-08-25 21:29:55,896] {base_executor.py:91} INFO - Adding to queue: 
['airflow', 'tasks', 'run', 'test_module', 'run_python', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp_nuoic9m']
   [2022-08-25 21:30:00,665] {local_executor.py:79} INFO - QueuedLocalWorker 
running ['airflow', 'tasks', 'run', 'test_module', 'run_python', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp_nuoic9m']
   [2022-08-25 21:30:00,679] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 1 | succeeded: 0 | running: 1 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 1
   [2022-08-25 21:30:00,695] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dags/test_module.py
   [2022-08-25 21:30:00,759] {task_command.py:371} INFO - Running 
<TaskInstance: test_module.run_python backfill__2021-12-01T00:00:00+00:00 
[queued]> on host 75512ab5e882
   [2022-08-25 21:30:05,686] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 1 | succeeded: 1 | running: 0 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 1
   [2022-08-25 21:30:05,709] {base_executor.py:91} INFO - Adding to queue: 
['airflow', 'tasks', 'run', 'test_module', 'test', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp3w9pm1jj']
   [2022-08-25 21:30:10,659] {local_executor.py:79} INFO - QueuedLocalWorker 
running ['airflow', 'tasks', 'run', 'test_module', 'test', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp3w9pm1jj']
   [2022-08-25 21:30:10,668] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 0 | succeeded: 1 | running: 1 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 0
   [2022-08-25 21:30:10,693] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dags/test_module.py
   [2022-08-25 21:30:10,765] {task_command.py:371} INFO - Running 
<TaskInstance: test_module.test backfill__2021-12-01T00:00:00+00:00 [queued]> 
on host 75512ab5e882
   [2022-08-25 21:30:15,678] {dagrun.py:564} INFO - Marking run <DagRun 
test_module @ 2021-12-01T00:00:00+00:00: backfill__2021-12-01T00:00:00+00:00, 
externally triggered: False> successful
   [2022-08-25 21:30:15,679] {dagrun.py:609} INFO - DagRun Finished: 
dag_id=test_module, execution_date=2021-12-01T00:00:00+00:00, 
run_id=backfill__2021-12-01T00:00:00+00:00, run_start_date=2022-08-25 
21:29:55.815199+00:00, run_end_date=2022-08-25 21:30:15.679256+00:00, 
run_duration=19.864057, state=success, external_trigger=False, 
run_type=backfill, data_interval_start=2021-12-01T00:00:00+00:00, 
data_interval_end=2021-12-02T00:00:00+00:00, dag_hash=None
   [2022-08-25 21:30:15,680] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 1 of 1 | tasks waiting: 0 | succeeded: 2 | running: 0 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 0
   [2022-08-25 21:30:15,684] {local_executor.py:390} INFO - Shutting down 
LocalExecutor; waiting for running tasks to finish.  Signal again if you don't 
want to wait.
   [2022-08-25 21:30:15,829] {backfill_job.py:879} INFO - Backfill done. 
Exiting.
   ```
   
   Second run of the backfill dagrun with the flags `--reset-dagruns --yes`. 
There is a warning about start_date is not set.
   
   ```
   astro@75512ab5e882:/usr/local/airflow$ airflow dags backfill -s 2021-12-01 
-e 2021-12-01 --reset-dagruns --yes test_module
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528: 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
     option = self._get_environment_variables(deprecated_key, 
deprecated_section, key, section)
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
   
/usr/local/lib/python3.9/site-packages/airflow/cli/commands/dag_command.py:57 
PendingDeprecationWarning: --ignore-first-depends-on-past is deprecated as the 
value is always set to True
   [2022-08-25 21:30:46,895] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dag
   /usr/local/lib/python3.9/site-packages/airflow/configuration.py:528 
DeprecationWarning: The sql_alchemy_conn option in [core] has been moved to the 
sql_alchemy_conn option in [database] - the old setting has been used, but 
please update your config.
   [2022-08-25 21:30:46,996] {executor_loader.py:105} INFO - Loaded executor: 
LocalExecutor
   [2022-08-25 21:30:47,275] {base_executor.py:91} INFO - Adding to queue: 
['airflow', 'tasks', 'run', 'test_module', 'run_python', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp3s_3bn80']
   [2022-08-25 21:30:52,010] {local_executor.py:79} INFO - QueuedLocalWorker 
running ['airflow', 'tasks', 'run', 'test_module', 'run_python', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmp3s_3bn80']
   [2022-08-25 21:30:52,029] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 1 | succeeded: 0 | running: 1 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 1
   [2022-08-25 21:30:52,045] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dags/test_module.py
   [2022-08-25 21:30:52,140] {task_command.py:371} INFO - Running 
<TaskInstance: test_module.run_python backfill__2021-12-01T00:00:00+00:00 
[queued]> on host 75512ab5e882
   [2022-08-25 21:30:57,028] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 1 | succeeded: 1 | running: 0 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 1
   [2022-08-25 21:30:57,048] {base_executor.py:91} INFO - Adding to queue: 
['airflow', 'tasks', 'run', 'test_module', 'test', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmprxg7g5o8']
   [2022-08-25 21:31:02,024] {local_executor.py:79} INFO - QueuedLocalWorker 
running ['airflow', 'tasks', 'run', 'test_module', 'test', 
'backfill__2021-12-01T00:00:00+00:00', '--ignore-depends-on-past', '--local', 
'--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/test_module.py', 
'--cfg-path', '/tmp/tmprxg7g5o8']
   [2022-08-25 21:31:02,032] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 0 of 1 | tasks waiting: 0 | succeeded: 1 | running: 1 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 0
   [2022-08-25 21:31:02,085] {dagbag.py:508} INFO - Filling up the DagBag from 
/usr/local/airflow/dags/test_module.py
   [2022-08-25 21:31:02,178] {task_command.py:371} INFO - Running 
<TaskInstance: test_module.test backfill__2021-12-01T00:00:00+00:00 [queued]> 
on host 75512ab5e882
   [2022-08-25 21:31:07,039] {dagrun.py:564} INFO - Marking run <DagRun 
test_module @ 2021-12-01 00:00:00+00:00: backfill__2021-12-01T00:00:00+00:00, 
externally triggered: False> successful
   [2022-08-25 21:31:07,039] {dagrun.py:609} INFO - DagRun Finished: 
dag_id=test_module, execution_date=2021-12-01 00:00:00+00:00, 
run_id=backfill__2021-12-01T00:00:00+00:00, run_start_date=None, 
run_end_date=2022-08-25 21:31:07.039737+00:00, run_duration=None, 
state=success, external_trigger=False, run_type=backfill, 
data_interval_start=2021-12-01 00:00:00+00:00, data_interval_end=2021-12-02 
00:00:00+00:00, dag_hash=None
   [2022-08-25 21:31:07,040] {dagrun.py:795} WARNING - Failed to record 
duration of <DagRun test_module @ 2021-12-01 00:00:00+00:00: 
backfill__2021-12-01T00:00:00+00:00, externally triggered: False>: start_date 
is not set.
   [2022-08-25 21:31:07,040] {backfill_job.py:367} INFO - [backfill progress] | 
finished run 1 of 1 | tasks waiting: 0 | succeeded: 2 | running: 0 | failed: 0 
| skipped: 0 | deadlocked: 0 | not ready: 0
   [2022-08-25 21:31:07,043] {local_executor.py:390} INFO - Shutting down 
LocalExecutor; waiting for running tasks to finish.  Signal again if you don't 
want to wait.
   [2022-08-25 21:31:07,177] {backfill_job.py:879} INFO - Backfill done. 
Exiting.
   ```
   
   ### What you think should happen instead
   
   When the BackfillJob fetches the dagrun, it will also need to set the start 
date.
   
   It can be done right after setting the run variable. 
([source](https://github.com/apache/airflow/blob/main/airflow/jobs/backfill_job.py#L310-L320))
   
   ### How to reproduce
   
   Run the backfill command first without `--reset-dagruns --yes` flags.
   
   ```
   airflow dags backfill -s 2021-12-01 -e 2021-12-01 test_module
   ```
   
   Run the backfill command with the `--reset-dagruns --yes` flags.
   
   ```
   airflow dags backfill -s 2021-12-01 -e 2021-12-01 --reset-dagruns --yes 
test_module
   ```
   
   ### Operating System
   
   N/A
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Astronomer
   
   ### Deployment details
   
   _No response_
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] 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