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/08/29 18:41:17 UTC

[GitHub] [airflow] wolfier opened a new issue, #26044: Backfill dagrun mistakenly evaluated as deadlocked

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

   ### Apache Airflow version
   
   Other Airflow 2 version
   
   ### What happened
   
   I used a bash operator to run a backfill command. The dagrun was marked as failed and was alerted for a deadlock even though the task instances themselves were still ran normally. This happens occasionally.
   
   ```
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {dagrun.py:585} ERROR - Deadlock; marking run <DagRun load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental @ 2022-08-15 08:00:00+00:00: backfill__2022-08-15T08:00:00+00:00, externally triggered: False> failed
   ...
   [2022-08-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 0 | succeeded: 4 | running: 0 | failed: 0 | skipped: 5 | deadlocked: 0 | not ready: 0
   [2022-08-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {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-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {backfill_job.py:879} INFO - Backfill done. Exiting.
   ```
   
   Here is full backfill log.
   
   ```
   [2022-08-23, 10:54:00 UTC] {subprocess.py:74} INFO - Running command: ['bash', '-c', 'cd $AIRFLOW_HOME && airflow dags backfill -s "2022-08-15 00:00:00" -e "2022-08-16 00:00:00"  -c \'{"start_val":"1","end_val":"4"}\' --rerun-failed-tasks --reset-dagruns --yes load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental']
   ...
   [2022-08-23, 10:54:21 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:21 UTC] {task_command.py:371} INFO - Running <TaskInstance: load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental.source.extract_withdrawals_venmo_withdrawal_aud_incremental_load backfill__2022-08-15T08:00:00+00:00 [queued]> on host a8870cb5a3e0
   [2022-08-23, 10:54:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:19 UTC] {local_executor.py:79} INFO - QueuedLocalWorker running ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'source.extract_withdrawals_venmo_withdrawal_aud_incremental_load', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmp92x61y3k']
   [2022-08-23, 10:54:24 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:24 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:29 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:29 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:34 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:34 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:39 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:39 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:44 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:44 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:49 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:49 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:54 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:54 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 0 of 1 | tasks waiting: 8 | succeeded: 0 | running: 1 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {dagrun.py:585} ERROR - Deadlock; marking run <DagRun load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental @ 2022-08-15 08:00:00+00:00: backfill__2022-08-15T08:00:00+00:00, externally triggered: False> failed
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {dagrun.py:609} INFO - DagRun Finished: dag_id=load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental, execution_date=2022-08-15 08:00:00+00:00, run_id=backfill__2022-08-15T08:00:00+00:00, run_start_date=None, run_end_date=2022-08-23 10:54:59.121952+00:00, run_duration=None, state=failed, external_trigger=False, run_type=backfill, data_interval_start=2022-08-15 08:00:00+00:00, data_interval_end=2022-08-16 08:00:00+00:00, dag_hash=None
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {dagrun.py:795} WARNING - Failed to record duration of <DagRun load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental @ 2022-08-15 08:00:00+00:00: backfill__2022-08-15T08:00:00+00:00, externally triggered: False>: start_date is not set.
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 8 | succeeded: 1 | running: 0 | failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 8
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {base_executor.py:91} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'destination.post_marker_staging_withdrawals_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmpd1nq6xe2']
   [2022-08-23, 10:54:59 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:54:59 UTC] {base_executor.py:91} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'destination.post_marker_fdg_pii_fact_aw_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmps6ah6zww']
   [2022-08-23, 10:55:04 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:04 UTC] {local_executor.py:79} INFO - QueuedLocalWorker running ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'destination.post_marker_fdg_pii_fact_aw_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmps6ah6zww']
   [2022-08-23, 10:55:04 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:04 UTC] {local_executor.py:79} INFO - QueuedLocalWorker running ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'destination.post_marker_staging_withdrawals_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmpd1nq6xe2']
   [2022-08-23, 10:55:04 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:04 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 3 | succeeded: 1 | running: 2 | failed: 0 | skipped: 3 | deadlocked: 0 | not ready: 3
   [2022-08-23, 10:55:06 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:06 UTC] {task_command.py:371} INFO - Running <TaskInstance: load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental.destination.post_marker_fdg_pii_fact_aw_venmo_withdrawal_aud backfill__2022-08-15T08:00:00+00:00 [queued]> on host a8870cb5a3e0
   [2022-08-23, 10:55:06 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:06 UTC] {task_command.py:371} INFO - Running <TaskInstance: load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental.destination.post_marker_staging_withdrawals_venmo_withdrawal_aud backfill__2022-08-15T08:00:00+00:00 [queued]> on host a8870cb5a3e0
   [2022-08-23, 10:55:09 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:09 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 1 | succeeded: 3 | running: 0 | failed: 0 | skipped: 5 | deadlocked: 0 | not ready: 1
   [2022-08-23, 10:55:09 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:09 UTC] {base_executor.py:91} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'post_execution.rotate_checkpoint_withdrawals_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmpkve4mv_q']
   [2022-08-23, 10:55:14 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:14 UTC] {local_executor.py:79} INFO - QueuedLocalWorker running ['airflow', 'tasks', 'run', 'load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental', 'post_execution.rotate_checkpoint_withdrawals_venmo_withdrawal_aud', 'backfill__2022-08-15T08:00:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/withdrawals_venmo_withdrawal_aud_jdbc_to_redshift_incremental_load.py', '--cfg-path', '/tmp/tmpkve4mv_q']
   [2022-08-23, 10:55:14 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:14 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 0 | succeeded: 3 | running: 1 | failed: 0 | skipped: 5 | deadlocked: 0 | not ready: 0
   [2022-08-23, 10:55:15 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:15 UTC] {task_command.py:371} INFO - Running <TaskInstance: load_withdrawals_venmo_withdrawal_aud_to_redshift_withdrawals_venmo_withdrawal_aud_incremental.post_execution.rotate_checkpoint_withdrawals_venmo_withdrawal_aud backfill__2022-08-15T08:00:00+00:00 [queued]> on host a8870cb5a3e0
   [2022-08-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {backfill_job.py:367} INFO - [backfill progress] | finished run 1 of 1 | tasks waiting: 0 | succeeded: 4 | running: 0 | failed: 0 | skipped: 5 | deadlocked: 0 | not ready: 0
   [2022-08-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {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-23, 10:55:19 UTC] {subprocess.py:92} INFO - [2022-08-23, 10:55:19 UTC] {backfill_job.py:879} INFO - Backfill done. Exiting.
   ```
   
   ### What you think should happen instead
   
   The DAG is not deadlocked but still somehow was still [evaluated as deadlocked](https://github.com/apache/airflow/blob/main/airflow/models/dagrun.py#L581-L589).
   
   
   
   ### How to reproduce
   
   _No response_
   
   ### 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: commits-unsubscribe@airflow.apache.org.apache.org

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


[GitHub] [airflow] ephraimbuddy commented on issue #26044: Backfill dagrun mistakenly evaluated as deadlocked

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

   @wolfier , can you share the dag you used to reproduce yours?


-- 
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] uranusjr closed issue #26044: Backfill dagrun mistakenly evaluated as deadlocked

Posted by GitBox <gi...@apache.org>.
uranusjr closed issue #26044: Backfill dagrun mistakenly evaluated as deadlocked
URL: https://github.com/apache/airflow/issues/26044


-- 
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] wolfier commented on issue #26044: Backfill dagrun mistakenly evaluated as deadlocked

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

   Unfortunately, I am unable to share the DAG itself but I can describe how the DAG is setup.
   
   There is a DAG that runs a BashOperator with the bash command of `airflow backfill`. The referenced DAG to backfill could be in any shape/size meaning this issue could happen to any DAG in the deployment.


-- 
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] ephraimbuddy commented on issue #26044: Backfill dagrun mistakenly evaluated as deadlocked

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

   So, using the dag at  #25353, I reproduced the deadlock. Not only in backfill but on normal run with mini scheduler turned off. 
   This case is similar to https://github.com/apache/airflow/issues/25200#issuecomment-1227870432, however, filtering the unfinished task does not solve it.
   cc: @ashb 


-- 
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