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/07/02 06:51:30 UTC

[GitHub] [airflow] huozhanfeng opened a new pull request #16768: Fix bug that log can't be shown when task runs failed

huozhanfeng opened a new pull request #16768:
URL: https://github.com/apache/airflow/pull/16768


   <!--
   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/
   -->
   ---
   The log can't be shown normally when the task runs failed. Users can only get useless logs as follows. #13692
   
   <pre>
   *** Log file does not exist: /home/airflow/airflow/logs/dag_id/task_id/2021-06-28T00:00:00+08:00/28.log
   *** Fetching from: http://:8793/log/dag_id/task_id/2021-06-28T00:00:00+08:00/28.log
   *** Failed to fetch log file from worker. Unsupported URL protocol 
   </pre>
   
   The root cause is that scheduler will overwrite the info into the task_instance table in db when tasks into failed.  Webserver can't get the right host of the task from task_instance, because the hostname info is lost in the progress.
   
   **^ 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] kaxil commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -573,6 +573,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
                 ti.start_date = simple_ti.start_date
                 ti.end_date = simple_ti.end_date
                 ti.try_number = simple_ti.try_number
+                ti.hostname = simple_ti.hostname

Review comment:
       probably calling `ti.refresh_from_db()` can do the trick instead of this?
   
   Have you tested with this fix?




-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-872763431


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -573,6 +573,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
                 ti.start_date = simple_ti.start_date
                 ti.end_date = simple_ti.end_date
                 ti.try_number = simple_ti.try_number
+                ti.hostname = simple_ti.hostname

Review comment:
       @kaxil hello, could you please help to take a look?




-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-872792607


   Can a test be added for this?


-- 
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] huozhanfeng removed a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng removed a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873851482


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Sure, but I meet a problem with UT.


-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873738826


   > Logic LGRM! A couple of minor comments/questions.
   > 
   > I wonder if/how we can refactor `SimpleTaskInstance` to prevent this kind of error in the future. Right now, if a field is added to `TaskInstace`, `SimpleTaskInstace` and `_execute_task_callbacks` needed to be checked to pass them on. This can be difficult to maintain.
   
   Agree with you, maybe we can raise an independent pr to solve it.


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"`.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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] huozhanfeng removed a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng removed a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-881374979


   hello, the failed checks seem to have nothing to do with this feature, I wonder how to merge it into the repo? 


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Find the one.  Looks like it related to airflow test command.
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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 change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -568,14 +568,11 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
             dag = dagbag.dags[simple_ti.dag_id]
             if simple_ti.task_id in dag.task_ids:
                 task = dag.get_task(simple_ti.task_id)
-                ti = TI(task, simple_ti.execution_date)
-                # Get properties needed for failure handling from SimpleTaskInstance.
-                ti.start_date = simple_ti.start_date
-                ti.end_date = simple_ti.end_date
-                ti.try_number = simple_ti.try_number
-                ti.state = simple_ti.state
-                ti.test_mode = self.UNIT_TEST_MODE
                 if request.is_failure_callback:
+                    ti = TI(task, simple_ti.execution_date)
+                    #TODO use simple_ti to improve performance here in the future
+                    ti.refresh_from_db()
+                    ti.test_mode = self.UNIT_TEST_MODE

Review comment:
       `handle_failure_with_callback` has an explicit `test_mode` argument, you can use that instead of hacking the value on like this




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   <pre>

##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   <pre>

##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       I add a config `dag_file_processor.UNIT_TEST_MODE = False` in UT as a workaround. It works well but I wonder whether it is normative.

##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   </pre>

##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the latest UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   </pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       > But I assume there is a reason it doesn’t save the task instance, although I do not know what. What would fail if you remove this change?
   
   I think so at the beginning, and I have tried to find the historical git log, nothing found. The UT always successful if remove this change although the logic is wrong because the task instance is not committed into DB/session in UT mode.




-- 
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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873916155


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Resolved it in a tricky way...


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: tests/dag_processing/test_processor.py
##########
@@ -643,6 +643,30 @@ def test_execute_on_failure_callbacks(self, mock_ti_handle_failure):
                 test_mode=conf.getboolean('core', 'unit_test_mode'),
             )
 
+    def test_failure_callbacks_should_not_drop_hostname(self):
+        dagbag = DagBag(dag_folder="/dev/null", include_examples=True, read_dags_from_db=False)
+        dag_file_processor = DagFileProcessor(dag_ids=[], log=mock.MagicMock())
+        dag_file_processor.UNIT_TEST_MODE = False
+
+        with create_session() as session:
+            session.query(TaskInstance).delete()

Review comment:
       Sure, thank you.




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   <pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: tests/dag_processing/test_processor.py
##########
@@ -643,6 +643,30 @@ def test_execute_on_failure_callbacks(self, mock_ti_handle_failure):
                 test_mode=conf.getboolean('core', 'unit_test_mode'),
             )
 
+    def test_failure_callbacks_should_not_drop_hostname(self):
+        dagbag = DagBag(dag_folder="/dev/null", include_examples=True, read_dags_from_db=False)
+        dag_file_processor = DagFileProcessor(dag_ids=[], log=mock.MagicMock())
+        with create_session() as session:
+            session.query(TaskInstance).delete()
+            dag = dagbag.get_dag('example_branch_operator')
+            task = dag.get_task(task_id='run_this_first')
+
+            ti = TaskInstance(task, DEFAULT_DATE, State.RUNNING)
+            ti.hostname = "test_hostname"
+
+            session.add(ti)
+            session.commit()

Review comment:
       Sure




-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873851482


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Sure, but I meet a problem with UT.


-- 
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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873738826


   > Logic LGRM! A couple of minor comments/questions.
   > 
   > I wonder if/how we can refactor `SimpleTaskInstance` to prevent this kind of error in the future. Right now, if a field is added to `TaskInstace`, `SimpleTaskInstace` and `_execute_task_callbacks` needed to be checked to pass them on. This can be difficult to maintain.
   
   Agree with you, maybe we can raise an independent pr to improve all related logic in the future.


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   <pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -573,6 +573,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
                 ti.start_date = simple_ti.start_date
                 ti.end_date = simple_ti.end_date
                 ti.try_number = simple_ti.try_number
+                ti.hostname = simple_ti.hostname

Review comment:
       Good idea, I will have a try.




-- 
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] github-actions[bot] commented on pull request #16768: Fix bug that log can't be shown when task runs failed

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


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the latest UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   </pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -573,6 +573,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
                 ti.start_date = simple_ti.start_date
                 ti.end_date = simple_ti.end_date
                 ti.try_number = simple_ti.try_number
+                ti.hostname = simple_ti.hostname

Review comment:
       Actually, I think  #16804 is the better way to solve this problem. Sometimes DB resource is tight.




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       > But I assume there is a reason it doesn’t save the task instance, although I do not know what. What would fail if you remove this change?
   
   I think so at the beginning, and I have tried to find the historical git log, nothing found. The UT always successful if remove this change although the logic is wrong because the task instance is not committed into DB/session in UT mode.




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. In the lasted UT, UT will raise `IndexError: list index out of range` with the code block `assert tis[0].hostname == "test_hostname"` if remove this change.  
   
   I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   https://github.com/apache/airflow/commit/aa01fab4906df3175e5e6589ec0095a5586279db
   </pre>




-- 
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] huozhanfeng removed a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng removed a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874571072


   Hello, should I rebase the PR to continue the CI? 
   
   `Please rebase or amend, and force push the PR to run full tests`


-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874022899


   The solution is good to me actually, not tricky at all but what it’s supposed to be done 🙂 Let’s see how the test suite feels about this.


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Yes, it can't write task_instance info into DB/session into ut in the original logic.




-- 
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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873916155


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Resolved it in a tricky way...


-- 
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] huozhanfeng removed a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng removed a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873851482


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Sure, but I meet a problem with UT.


-- 
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] github-actions[bot] commented on pull request #16768: Fix bug that log can't be shown when task runs failed

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


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

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 change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       But I assume there is a reason it doesn’t save the task instance, although I do not know what. What would fail if you remove 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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-872813414


   > Can a test be added for this?
   
   Sure, I will append one for it.


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -568,14 +568,11 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
             dag = dagbag.dags[simple_ti.dag_id]
             if simple_ti.task_id in dag.task_ids:
                 task = dag.get_task(simple_ti.task_id)
-                ti = TI(task, simple_ti.execution_date)
-                # Get properties needed for failure handling from SimpleTaskInstance.
-                ti.start_date = simple_ti.start_date
-                ti.end_date = simple_ti.end_date
-                ti.try_number = simple_ti.try_number
-                ti.state = simple_ti.state
-                ti.test_mode = self.UNIT_TEST_MODE
                 if request.is_failure_callback:
+                    ti = TI(task, simple_ti.execution_date)
+                    #TODO use simple_ti to improve performance here in the future
+                    ti.refresh_from_db()
+                    ti.test_mode = self.UNIT_TEST_MODE

Review comment:
       Sure, I will improve it now.




-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873769101


   Please see comments above: https://github.com/apache/airflow/pull/16768#discussion_r663619814


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/dag_processing/processor.py
##########
@@ -573,6 +573,7 @@ def _execute_task_callbacks(self, dagbag: DagBag, request: TaskCallbackRequest):
                 ti.start_date = simple_ti.start_date
                 ti.end_date = simple_ti.end_date
                 ti.try_number = simple_ti.try_number
+                ti.hostname = simple_ti.hostname

Review comment:
       Resolved it and let the `ti.refresh_from_db()` action only runs in the failure_callback block due to performance concern.




-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873916155






-- 
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 change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Is this related?

##########
File path: tests/dag_processing/test_processor.py
##########
@@ -643,6 +643,30 @@ def test_execute_on_failure_callbacks(self, mock_ti_handle_failure):
                 test_mode=conf.getboolean('core', 'unit_test_mode'),
             )
 
+    def test_failure_callbacks_should_not_drop_hostname(self):
+        dagbag = DagBag(dag_folder="/dev/null", include_examples=True, read_dags_from_db=False)
+        dag_file_processor = DagFileProcessor(dag_ids=[], log=mock.MagicMock())
+        with create_session() as session:
+            session.query(TaskInstance).delete()
+            dag = dagbag.get_dag('example_branch_operator')
+            task = dag.get_task(task_id='run_this_first')
+
+            ti = TaskInstance(task, DEFAULT_DATE, State.RUNNING)
+            ti.hostname = "test_hostname"
+
+            session.add(ti)
+            session.commit()

Review comment:
       Split this block into two `create_session()`s instead of using `commit()`.




-- 
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] kaxil merged pull request #16768: Fix bug that log can't be shown when task runs failed

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


   


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Yes, the original logic can't write task_instance info into DB/session in ut mode.




-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-885270047


   Awesome work, congrats on your first merged pull request!
   


-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874571072


   Hello, Should I rebase the PR to continue the CI? 
   
   `Please rebase or amend, and force push the PR to run full tests`


-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874045934


   > The solution is good to me actually, not tricky at all but what it’s supposed to be done 🙂 Let’s see how the test suite feels about this.
   
   That's great and thank you!


-- 
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 a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: tests/dag_processing/test_processor.py
##########
@@ -643,6 +643,30 @@ def test_execute_on_failure_callbacks(self, mock_ti_handle_failure):
                 test_mode=conf.getboolean('core', 'unit_test_mode'),
             )
 
+    def test_failure_callbacks_should_not_drop_hostname(self):
+        dagbag = DagBag(dag_folder="/dev/null", include_examples=True, read_dags_from_db=False)
+        dag_file_processor = DagFileProcessor(dag_ids=[], log=mock.MagicMock())
+        dag_file_processor.UNIT_TEST_MODE = False
+
+        with create_session() as session:
+            session.query(TaskInstance).delete()

Review comment:
       ```suggestion
   ```
   We have `clear_db_runs` in the setup method which clears TaskInstance as well, so we don't need this




-- 
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 pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874022899


   The solution is good to me actually, not tricky at all but what it’s supposed to be done 🙂 Let’s see how the test suite feels about this.


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       @uranusjr Find the one.  Looks like it's related to the airflow test command. I wonder whether it is reasonable to mix UT and test commands together here by using a test_mode variable because the test command doesn't want to save the state into DB/session but UT should save the state into DB/session. Do you have any idea about it?
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-881374979


   hello, the failed checks seem to have nothing to do with this feature, I wonder how to merge it into the repo? 


-- 
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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-874571072


   Hello, should I rebase the PR to continue the CI? 
   
   `Please rebase or amend, and force push the PR to run full tests`


-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-872813414


   > Can a test be added for this?
   
   Sure, I will try to append tests for it.


-- 
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] huozhanfeng commented on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng commented on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873916155


   > Please see comments above: [#16768 (comment)](https://github.com/apache/airflow/pull/16768#discussion_r663619814)
   
   Resolved it by using a tricky method...


-- 
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] huozhanfeng edited a comment on pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
huozhanfeng edited a comment on pull request #16768:
URL: https://github.com/apache/airflow/pull/16768#issuecomment-873738826


   > Logic LGRM! A couple of minor comments/questions.
   > 
   > I wonder if/how we can refactor `SimpleTaskInstance` to prevent this kind of error in the future. Right now, if a field is added to `TaskInstace`, `SimpleTaskInstace` and `_execute_task_callbacks` needed to be checked to pass them on. This can be difficult to maintain.
   
   Agree with you, maybe we can raise an independent pr to improve related logic in the future...


-- 
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 closed pull request #16768: Fix bug that log can't be shown when task runs failed

Posted by GitBox <gi...@apache.org>.
ephraimbuddy closed pull request #16768:
URL: https://github.com/apache/airflow/pull/16768


   


-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Find the one.  Looks like it's related to airflow test command.
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       Find the one...
   <pre>
   commit aa01fab4906df3175e5e6589ec0095a5586279db
   Author: Maxime Beauchemin <ma...@apache.org>
   Date:   Mon Jan 12 21:54:40 2015 +0000
   
       Improving the test command line subcommand
   
   <pre>




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       > But I assume there is a reason it doesn’t save the task instance, although I do not know what. What would fail if you remove this change?
   
   I think so at the beginning, and I have tried to find the historical git log, nothing found. The UT always successful if remove this change although the logic is wrong.




-- 
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] huozhanfeng commented on a change in pull request #16768: Fix bug that log can't be shown when task runs failed

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



##########
File path: airflow/models/taskinstance.py
##########
@@ -1495,8 +1495,7 @@ def handle_failure(
             except Exception:
                 self.log.exception('Failed to send email to: %s', task.email)
 
-        if not test_mode:
-            session.merge(self)
+        session.merge(self)

Review comment:
       I add a config `dag_file_processor.UNIT_TEST_MODE = False` in UT as a workaround. It works well but I wonder whether it is normative.




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