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/01/16 02:13:39 UTC

[GitHub] [airflow] houqp opened a new pull request #13712: Fix backfill crash on task retry or reschedule

houqp opened a new pull request #13712:
URL: https://github.com/apache/airflow/pull/13712


   When a retry happens, task key needs to be recorded with try number + 1 to avoid KeyError exception.


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

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



[GitHub] [airflow] houqp commented on a change in pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #13712:
URL: https://github.com/apache/airflow/pull/13712#discussion_r558765227



##########
File path: tests/test_utils/mock_executor.py
##########
@@ -69,6 +69,7 @@ def sort_by(item):
             for index in range(min((open_slots, len(sorted_queue)))):
                 (key, (_, _, _, ti)) = sorted_queue[index]
                 self.queued_tasks.pop(key)
+                ti._try_number += 1

Review comment:
       simulate taskinstance.check_and_change_state_before_execution




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

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



[GitHub] [airflow] houqp commented on a change in pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #13712:
URL: https://github.com/apache/airflow/pull/13712#discussion_r558765227



##########
File path: tests/test_utils/mock_executor.py
##########
@@ -69,6 +69,7 @@ def sort_by(item):
             for index in range(min((open_slots, len(sorted_queue)))):
                 (key, (_, _, _, ti)) = sorted_queue[index]
                 self.queued_tasks.pop(key)
+                ti._try_number += 1

Review comment:
       simulate try_number bump in `taskinstance.check_and_change_state_before_execution`




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

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



[GitHub] [airflow] houqp merged pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
houqp merged pull request #13712:
URL: https://github.com/apache/airflow/pull/13712


   


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

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



[GitHub] [airflow] kaxil commented on a change in pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #13712:
URL: https://github.com/apache/airflow/pull/13712#discussion_r559033498



##########
File path: tests/jobs/test_backfill_job.py
##########
@@ -683,6 +685,61 @@ def test_backfill_rerun_failed_tasks_without_flag(self):
         with self.assertRaises(AirflowException):
             job.run()
 
+    def test_backfill_retry_intermittent_failed_task(self):
+        dag = DAG(
+            dag_id='test_intermittent_failure_job',
+            start_date=DEFAULT_DATE,
+            schedule_interval="@daily",
+            default_args={
+                'retries': 2,
+                'retry_delay': datetime.timedelta(seconds=0),
+            },
+        )
+        task1 = DummyOperator(task_id="task1", dag=dag)
+        dag.clear()
+
+        executor = MockExecutor(parallelism=16)
+        executor.mock_task_results[
+            TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)
+        ] = State.UP_FOR_RETRY
+        executor.mock_task_results[
+            TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)
+        ] = State.UP_FOR_RETRY

Review comment:
       Shouldn't the first try be failed for it to go to try_number 2?




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #13712:
URL: https://github.com/apache/airflow/pull/13712#issuecomment-761552627


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest master at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


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

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



[GitHub] [airflow] houqp commented on a change in pull request #13712: Fix backfill crash on task retry or reschedule

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #13712:
URL: https://github.com/apache/airflow/pull/13712#discussion_r559045866



##########
File path: tests/jobs/test_backfill_job.py
##########
@@ -683,6 +685,61 @@ def test_backfill_rerun_failed_tasks_without_flag(self):
         with self.assertRaises(AirflowException):
             job.run()
 
+    def test_backfill_retry_intermittent_failed_task(self):
+        dag = DAG(
+            dag_id='test_intermittent_failure_job',
+            start_date=DEFAULT_DATE,
+            schedule_interval="@daily",
+            default_args={
+                'retries': 2,
+                'retry_delay': datetime.timedelta(seconds=0),
+            },
+        )
+        task1 = DummyOperator(task_id="task1", dag=dag)
+        dag.clear()
+
+        executor = MockExecutor(parallelism=16)
+        executor.mock_task_results[
+            TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)
+        ] = State.UP_FOR_RETRY
+        executor.mock_task_results[
+            TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)
+        ] = State.UP_FOR_RETRY

Review comment:
       If i understand your question correctly, setting it to failed will result in backfill job to exit immediately without retry.




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

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