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/04/20 14:03:36 UTC

[GitHub] [airflow] ashb opened a new pull request, #23119: Fix TI failure handling when task cannot be unmapped.

ashb opened a new pull request, #23119:
URL: https://github.com/apache/airflow/pull/23119

   At first glance this looks like a lot of un-related changed, but it is
   all related to handling errors in unmapping:
   
   - Ensure that SimpleTaskInstance (and thus the Zombie callback) knows
     about map_index, and simplify the code for SimpleTaskInstance -- no
     need for properties, just attributes works.
   
   - Be able to create a TaskFail from a TI, not a Task.
   
     This is so that we can create the TaskFail with the mapped task so we
     can delay unmapping the task in TI.handle_failure as long as possible.
   
   - Change email_alert and get_email_subject_content to take the task so
     we can pass the unmapped Task around.
   
   Fixes #23107
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.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] uranusjr commented on a diff in pull request #23119: Fix TI failure handling when task cannot be unmapped.

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #23119:
URL: https://github.com/apache/airflow/pull/23119#discussion_r855664363


##########
airflow/models/taskinstance.py:
##########
@@ -1901,24 +1903,15 @@ def handle_failure(
         if not test_mode:
             self.refresh_from_db(session)
 
-        task = self.task.unmap()
         self.end_date = timezone.utcnow()
         self.set_duration()
-        Stats.incr(f'operator_failures_{task.task_type}', 1, 1)
+        Stats.incr(f'operator_failures_{self.task.task_type}')

Review Comment:
   What does this change?



-- 
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] jedcunningham commented on a diff in pull request #23119: Fix TI failure handling when task cannot be unmapped.

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #23119:
URL: https://github.com/apache/airflow/pull/23119#discussion_r854200462


##########
airflow/models/taskinstance.py:
##########
@@ -1934,20 +1927,26 @@ def handle_failure(
         # only mark task instance as FAILED if the next task instance
         # try_number exceeds the max_tries ... or if force_fail is truthy
 
+        task = None
+        try:
+            task = self.task.unmap()
+        except Exception:
+            self.log.error("Unable to unmap task, can't determine if we need to send an alert email ot not")

Review Comment:
   ```suggestion
               self.log.error("Unable to unmap task, can't determine if we need to send an alert email or not")
   ```



-- 
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 commented on a diff in pull request #23119: Fix TI failure handling when task cannot be unmapped.

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #23119:
URL: https://github.com/apache/airflow/pull/23119#discussion_r855665917


##########
airflow/dag_processing/processor.py:
##########
@@ -604,7 +604,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
             if simple_ti.task_id in dag.task_ids:
                 task = dag.get_task(simple_ti.task_id)
                 if request.is_failure_callback:
-                    ti = TI(task, run_id=simple_ti.run_id)
+                    ti = TI(task, run_id=simple_ti.run_id, map_index=simple_ti.map_index)

Review Comment:
   Perhaps we should revive #19242, it can make the TI <-> SimpleTI conversions more future-proof.



-- 
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 commented on a diff in pull request #23119: Fix TI failure handling when task cannot be unmapped.

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #23119:
URL: https://github.com/apache/airflow/pull/23119#discussion_r855664618


##########
airflow/models/taskinstance.py:
##########
@@ -1934,20 +1927,26 @@ def handle_failure(
         # only mark task instance as FAILED if the next task instance
         # try_number exceeds the max_tries ... or if force_fail is truthy
 
+        task = None
+        try:
+            task = self.task.unmap()
+        except Exception:
+            self.log.error("Unable to unmap task, can't determine if we need to send an alert email or not")

Review Comment:
   Should we log the traceback here with `exception()` instead?



-- 
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] ashb merged pull request #23119: Fix TI failure handling when task cannot be unmapped.

Posted by GitBox <gi...@apache.org>.
ashb merged PR #23119:
URL: https://github.com/apache/airflow/pull/23119


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