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 2021/09/15 23:54:04 UTC

[GitHub] [airflow] dima-asana opened a new issue #18285: Retry count off by one on tasks that are cleared via clear_task_instances

dima-asana opened a new issue #18285:
URL: https://github.com/apache/airflow/issues/18285


   ### Apache Airflow version
   
   2.1.2
   
   ### Operating System
   
   any
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Docker-Compose
   
   ### Deployment details
   
   _No response_
   
   ### What happened
   
   I think there's an off by one error at https://github.com/apache/airflow/blob/ed10edd20b3c656fe1cd9b1c17468c0026f075e2/airflow/models/taskinstance.py#L198.
   
   To reproduce, 
   1. create a task with 2 retries that always fails twice and succeeds on the 3rd try
   2. run this task
   3. clear this task
   4. the task will only retry 1 time instead of 2, so it will fail
   
   Suppose we have a task that has 2 retries, and always fails twice and succeeds on the 3rd try.
   
   max_tries is set to task.retries in refresh_from_task, so it's set to 2.
   at the start, try_number=0 and we don't need to do any checks to retry
   on first fail, try_number=1, max_tries=2, so we retry again
   on second fail, try_number=2, max_tries=2, so we retry again
   the task works.  this is as intended.
   
   But it doesn't interact well with task clearing.  Let's say we clear this task next, via 
   at this point try_number=3 and max_tries=2.
   this sets max_tries = 3 + 2 - 1 = 4 through code at https://github.com/apache/airflow/blob/ed10edd20b3c656fe1cd9b1c17468c0026f075e2/airflow/models/taskinstance.py#L198
   
   on next try, try_number=3 and max_tries=4, task fails.
   on first fail, try_number=4 and max_tries = 4, so we retry
   on second fail, try_number=5, se we don't do a 2nd retry.
   
   ### What you expected to happen
   
   _No response_
   
   ### How to reproduce
   
   _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: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] bhavaniravi commented on issue #18285: Retry count off by one on tasks that are cleared via clear_task_instances

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


   Hey @dima-asana 
   
   I tried to recreate the issue in 2.1.3 with the following DAGs, but it was working as expected.
   
   ```
   from airflow import DAG
   from airflow.operators.python import PythonOperator
   from airflow.utils.dates import days_ago
   
   args = {
       'owner': 'airflow',
   }
   
   def fail_for_3(ti):
       if ti.try_number % 3 == 0:
          print (f"yipppe {ti.try_number} {ti.max_tries}")
       else:        
           print (f"oh ow!!! {ti.try_number} {ti.max_tries}")
           exit(0)
   
   with DAG(
       dag_id='test_retries',
       default_args=args,
       schedule_interval=None,
       start_date=days_ago(2),
   ) as dag:
   
       run_this = PythonOperator(
           retries=2,
           task_id='fail_gt_3',
           python_callable=fail_for_3
       )
   ```
   
   ## Log Summary
   
   | Attempt  |  try_number  | max_retry  |  Status  |
   |---|---|---|---|
   | 1/3 | 1 | 2 | Retry |
   2/4|2|3|Retry
   3/5|3|4|Success
   4/6|4|5|Retry
   5/7|5|6|Retry
   6/8|6|7|Success
   
   <details>
     <summary>Detailed logs</summary>
   
   ```
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/1.log
   [2021-09-16 18:02:07,963] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:07,970] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:07,970] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:07,970] {taskinstance.py:1233} INFO - Starting attempt 1 of 3
   [2021-09-16 18:02:07,970] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:07,979] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:02:07,980] {standard_task_runner.py:52} INFO - Started process 2294 to run task
   [2021-09-16 18:02:07,985] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '20', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpds9jv98a', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmp3hbcc3_7']
   [2021-09-16 18:02:07,986] {standard_task_runner.py:77} INFO - Job 20: Subtask fail_gt_3
   [2021-09-16 18:02:08,016] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:02:08,039] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:02:08,039] {logging_mixin.py:109} INFO - oh ow!!! 1 2
   [2021-09-16 18:02:08,302] {local_task_job.py:154} INFO - Task exited with return code 1
   [2021-09-16 18:02:08,314] {taskinstance.py:1271} INFO - Marking task as UP_FOR_RETRY. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123207, end_date=20210916T123208
   [2021-09-16 18:02:08,332] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/2.log
   [2021-09-16 18:02:18,509] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:18,516] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:18,516] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:18,516] {taskinstance.py:1233} INFO - Starting attempt 2 of 4
   [2021-09-16 18:02:18,516] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:18,525] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:02:18,527] {standard_task_runner.py:52} INFO - Started process 2341 to run task
   [2021-09-16 18:02:18,531] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '21', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpbujghrqn', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpeggff9r5']
   [2021-09-16 18:02:18,533] {standard_task_runner.py:77} INFO - Job 21: Subtask fail_gt_3
   [2021-09-16 18:02:18,562] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:02:18,583] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:02:18,583] {logging_mixin.py:109} INFO - oh ow!!! 2 3
   [2021-09-16 18:02:18,856] {local_task_job.py:154} INFO - Task exited with return code 1
   [2021-09-16 18:02:18,868] {taskinstance.py:1271} INFO - Marking task as UP_FOR_RETRY. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123218, end_date=20210916T123218
   [2021-09-16 18:02:18,888] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/3.log
   [2021-09-16 18:02:26,811] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:26,820] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:02:26,820] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:26,820] {taskinstance.py:1233} INFO - Starting attempt 3 of 5
   [2021-09-16 18:02:26,820] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:02:26,830] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:02:26,833] {standard_task_runner.py:52} INFO - Started process 2370 to run task
   [2021-09-16 18:02:26,839] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '22', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpd7x_9qqk', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmptqi8xxkn']
   [2021-09-16 18:02:26,842] {standard_task_runner.py:77} INFO - Job 22: Subtask fail_gt_3
   [2021-09-16 18:02:26,883] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:02:26,912] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:02:26,913] {logging_mixin.py:109} INFO - yipppe 3 4
   [2021-09-16 18:02:26,913] {python.py:152} INFO - Done. Returned value was: None
   [2021-09-16 18:02:26,918] {taskinstance.py:1271} INFO - Marking task as SUCCESS. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123226, end_date=20210916T123226
   [2021-09-16 18:02:26,940] {local_task_job.py:154} INFO - Task exited with return code 0
   [2021-09-16 18:02:26,956] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/4.log
   [2021-09-16 18:04:51,953] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:04:51,959] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:04:51,959] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:04:51,959] {taskinstance.py:1233} INFO - Starting attempt 4 of 6
   [2021-09-16 18:04:51,959] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:04:51,967] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:04:51,969] {standard_task_runner.py:52} INFO - Started process 2728 to run task
   [2021-09-16 18:04:51,974] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '23', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmp2bpp7zbf', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpe_0vxo4b']
   [2021-09-16 18:04:51,976] {standard_task_runner.py:77} INFO - Job 23: Subtask fail_gt_3
   [2021-09-16 18:04:52,006] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:04:52,030] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:04:52,031] {logging_mixin.py:109} INFO - oh ow!!! 4 5
   [2021-09-16 18:04:52,328] {local_task_job.py:154} INFO - Task exited with return code 1
   [2021-09-16 18:04:52,355] {taskinstance.py:1271} INFO - Marking task as UP_FOR_RETRY. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123451, end_date=20210916T123452
   [2021-09-16 18:04:52,373] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/5.log
   [2021-09-16 18:05:00,401] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:05:00,409] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:05:00,409] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:05:00,409] {taskinstance.py:1233} INFO - Starting attempt 5 of 7
   [2021-09-16 18:05:00,410] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:05:00,418] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:05:00,420] {standard_task_runner.py:52} INFO - Started process 2759 to run task
   [2021-09-16 18:05:00,425] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '24', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpecy6cjcp', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpy_y4k09d']
   [2021-09-16 18:05:00,427] {standard_task_runner.py:77} INFO - Job 24: Subtask fail_gt_3
   [2021-09-16 18:05:00,459] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:05:00,482] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:05:00,483] {logging_mixin.py:109} INFO - oh ow!!! 5 6
   [2021-09-16 18:05:00,731] {local_task_job.py:154} INFO - Task exited with return code 1
   [2021-09-16 18:05:00,744] {taskinstance.py:1271} INFO - Marking task as UP_FOR_RETRY. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123500, end_date=20210916T123500
   [2021-09-16 18:05:00,760] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   *** Reading local file: /Users/bhavaniravi/airflow-2.1.3/logs/test_retries/fail_gt_3/2021-09-16T12:32:04.244503+00:00/6.log
   [2021-09-16 18:05:15,121] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:05:15,129] {taskinstance.py:1034} INFO - Dependencies all met for <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [queued]>
   [2021-09-16 18:05:15,129] {taskinstance.py:1232} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:05:15,130] {taskinstance.py:1233} INFO - Starting attempt 6 of 8
   [2021-09-16 18:05:15,130] {taskinstance.py:1234} INFO - 
   --------------------------------------------------------------------------------
   [2021-09-16 18:05:15,140] {taskinstance.py:1253} INFO - Executing <Task(PythonOperator): fail_gt_3> on 2021-09-16 12:32:04.244503+00:00
   [2021-09-16 18:05:15,143] {standard_task_runner.py:52} INFO - Started process 2812 to run task
   [2021-09-16 18:05:15,148] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test_retries', 'fail_gt_3', 'manual__2021-09-16T12:32:04.244503+00:00', '--job-id', '25', '--raw', '--subdir', 'DAGS_FOLDER/18285_task_retries.py', '--cfg-path', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmpl2qoklyx', '--error-file', '/var/folders/yl/7bhj2fqd3jb17g9pz4vw8ddc0000gn/T/tmp7w9w8gad']
   [2021-09-16 18:05:15,150] {standard_task_runner.py:77} INFO - Job 25: Subtask fail_gt_3
   [2021-09-16 18:05:15,184] {logging_mixin.py:109} INFO - Running <TaskInstance: test_retries.fail_gt_3 manual__2021-09-16T12:32:04.244503+00:00 [running]> on host 1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa
   [2021-09-16 18:05:15,206] {taskinstance.py:1408} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test_retries
   AIRFLOW_CTX_TASK_ID=fail_gt_3
   AIRFLOW_CTX_EXECUTION_DATE=2021-09-16T12:32:04.244503+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-09-16T12:32:04.244503+00:00
   [2021-09-16 18:05:15,207] {logging_mixin.py:109} INFO - yipppe 6 7
   [2021-09-16 18:05:15,207] {python.py:152} INFO - Done. Returned value was: None
   [2021-09-16 18:05:15,210] {taskinstance.py:1271} INFO - Marking task as SUCCESS. dag_id=test_retries, task_id=fail_gt_3, execution_date=20210916T123204, start_date=20210916T123515, end_date=20210916T123515
   [2021-09-16 18:05:15,248] {local_task_job.py:154} INFO - Task exited with return code 0
   [2021-09-16 18:05:15,266] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   
   ```
   </details>
   
   


-- 
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] potiuk closed issue #18285: Retry count off by one on tasks that are cleared via clear_task_instances

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #18285:
URL: https://github.com/apache/airflow/issues/18285


   


-- 
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] boring-cyborg[bot] commented on issue #18285: Retry count off by one on tasks that are cleared via clear_task_instances

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #18285:
URL: https://github.com/apache/airflow/issues/18285#issuecomment-920468771


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


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