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/04/26 17:46:36 UTC

[GitHub] [airflow] ephraimbuddy opened a new pull request #15537: Fix on_failure_callback when task receive SIGKILL

ephraimbuddy opened a new pull request #15537:
URL: https://github.com/apache/airflow/pull/15537


   This PR fixes a case where a task would not call the on_failure_callback
   when there's a case of OOM. The issue was that task PID was being set
   at the wrong place and the local task job heartbeat was not checking the
   correct PID of the process runner and task.
   
   Now, instead of setting the task PID in check_and_change_state_before_execution method,
   it's now set correctly at the _run_raw_task method
   
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/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/master/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.

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



[GitHub] [airflow] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > In that example I don't see pid 30960 anywhere -- do you know what process it was it?
   > 
   > I suspect this might be to do with impersonation, rather than anything hive in particular.
   
   @ashb Sorry for my mistake of copying the wrong logs. I have updated it by using a new airflow task.  Agree with you about your suspicion. In my env, the shell-type task can run normally, and meanwhile, tasks that have subprocess all fail to run.


-- 
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] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > In that example I don't see pid 30960 anywhere -- do you know what process it was it?
   > 
   > I suspect this might be to do with impersonation, rather than anything hive in particular.
   
   @ashb Sorry for my mistake of copying the wrong logs. I have updated it by using a new airflow task.  Agree with you about your suspicion. In my env, the shell-type task can run normally, and meanwhile, tasks that have subprocess all fail to run.


-- 
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] ephraimbuddy commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   Hi @huozhanfeng, I'm taking a look at fixing this but would like it if you can give me a reproducible step using Bash or python operator.
   


-- 
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] huozhanfeng commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   The PR will cause dags that use hive operations to fail because they can't get the right PID.
   
   the hive task process info like these 
   1、airflow  20391 20170 11 17:41 ?        00:00:00 airflow task supervisor: ['airflow...
   2、root     20394 20391  0 17:41 ?        00:00:00 sudo -E -H -u user airflow tasks run ...
   3、user+ 20396 20394 61 17:41 ?        00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run...
   4、user+ 20407 20396 99 17:41 ?        00:00:04 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4...
   
   


-- 
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] ashb commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   In that example I don't see pid 30960 anywhere -- do you know what process it was it?
   
   I suspect this might be to do with impersonation, rather than anything hive in particular.
   


-- 
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] ephraimbuddy commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > Can `test_process_kill_call_on_failure_callback` and `test_process_sigkill_call_on_failure_callback` be combined with `pytest.mark.parametrize()`? They look much too similar to me. (Potentially `test_mark_success_on_success_callback` as well.)
   
   Have parameterized them except `test_mark_success_on_success_callback` because of the many `if` statements I had to deal with


-- 
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] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   The PR will cause dags that use hive operations to fail because they can't get the right PID.
   
   the hive task processes info like these 
   1、airflow  30539  1721  9 16:31 ?        00:00:00 airflow task supervisor: ['airflow', 'tasks', 'run', 'GAEA_hive_task_test'...
   2、root     30542 30539  0 16:31 ?        00:00:00 sudo -E -H -u user airflow tasks run GAEA_hive_task_test...
   3、user+ 30544 30542 48 16:31 ?        00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run GAEA_hive_task_test...
   4、user+ 30555 30544 99 16:31 ?        00:00:07 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4Stack=true ...
   
   The error log as follows:
   
   [2021-06-19 16:31:52,619] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   [2021-06-19 16:31:54,304] {local_task_job.py:193} WARNING - Recorded pid 30544 does not match the current pid 30542
   [2021-06-19 16:31:54,310] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30542
   [2021-06-19 16:31:54,326] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-06-19 16:31:54,326] {logging_mixin.py:104} INFO - Killing the Hive job
   [2021-06-19 16:32:54,355] {process_utils.py:113} WARNING - process psutil.Process(pid=30542, name='sudo', status='sleeping', started='16:31:48') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,357] {process_utils.py:113} WARNING - process psutil.Process(pid=30555, name='java', status='zombie', started='16:31:50') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,357] {process_utils.py:113} WARNING - process psutil.Process(pid=30544, name='airflow', status='sleeping', started='16:31:48') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,375] {process_utils.py:66} INFO - Process psutil.Process(pid=30542, name='sudo', status='terminated', exitcode=<Negsignal.SIGKILL: -9>, started='16:31:48') (30542) terminated with exit code Negsignal.SIGKILL
   [2021-06-19 16:32:54,403] {process_utils.py:66} INFO - Process psutil.Process(pid=30555, name='java', status='terminated', started='16:31:50') (30555) terminated with exit code None
   [2021-06-19 16:32:54,403] {process_utils.py:66} INFO - Process psutil.Process(pid=30544, name='airflow', status='terminated', started='16:31:48') (30544) terminated with exit code None
   [2021-06-19 16:32:54,404] {standard_task_runner.py:130} ERROR - Job 1354 was killed before it finished (likely due to running out of memory)
   
   
   The demo with hive operation runs success after deleting all the code of this PR. So please consider improving the PR or repeal 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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   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] ashb merged pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   


-- 
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] huozhanfeng commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > In that example I don't see pid 30960 anywhere -- do you know what process it was it?
   > 
   > I suspect this might be to do with impersonation, rather than anything hive in particular.
   
   Sorry for my mistake of copying the wrong logs. I have updated it by using a new airflow task.  Agree with you about your suspicion. In my env, the shell-type task can run normally, and meanwhile, tasks that have subprocess all fail to run.


-- 
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] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   The PR will cause dags that use hive operations to fail because they can't get the right PID.
   
   the hive task process info like these 
   1、airflow  20391 20170 11 17:41 ?        00:00:00 airflow task supervisor: ['airflow...
   2、root     20394 20391  0 17:41 ?        00:00:00 sudo -E -H -u user airflow tasks run ...
   3、user+ 20396 20394 61 17:41 ?        00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run...
   4、user+ 20407 20396 99 17:41 ?        00:00:04 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4...
   
   The error log as follows:
   
   [2021-06-15 16:40:10,222] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   [2021-06-15 16:40:11,871] {local_task_job.py:193} WARNING - Recorded pid 30962 does not match the current pid 30960
   [2021-06-15 16:40:11,877] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30960
   [2021-06-15 16:40:11,901] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-06-15 16:40:11,902] {logging_mixin.py:104} INFO - Killing the Hive job


-- 
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] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   The PR will cause dags that use hive operations to fail because they can't get the right PID.
   
   the hive task processes info like these 
   ```
   1、airflow  30539  1721  9 16:31 ?        00:00:00 airflow task supervisor: ['airflow', 'tasks', 'run', 'GAEA_hive_task_test'...
   2、root     30542 30539  0 16:31 ?        00:00:00 sudo -E -H -u user airflow tasks run GAEA_hive_task_test...
   3、user+ 30544 30542 48 16:31 ?        00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run GAEA_hive_task_test...
   4、user+ 30555 30544 99 16:31 ?        00:00:07 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4Stack=true ...
   ```
   
   The error log as follows:
   ```
   [2021-06-19 16:31:52,619] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   [2021-06-19 16:31:54,304] {local_task_job.py:193} WARNING - Recorded pid 30544 does not match the current pid 30542
   [2021-06-19 16:31:54,310] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30542
   [2021-06-19 16:31:54,326] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-06-19 16:31:54,326] {logging_mixin.py:104} INFO - Killing the Hive job
   [2021-06-19 16:32:54,355] {process_utils.py:113} WARNING - process psutil.Process(pid=30542, name='sudo', status='sleeping', started='16:31:48') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,357] {process_utils.py:113} WARNING - process psutil.Process(pid=30555, name='java', status='zombie', started='16:31:50') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,357] {process_utils.py:113} WARNING - process psutil.Process(pid=30544, name='airflow', status='sleeping', started='16:31:48') did not respond to SIGTERM. Trying SIGKILL
   [2021-06-19 16:32:54,375] {process_utils.py:66} INFO - Process psutil.Process(pid=30542, name='sudo', status='terminated', exitcode=<Negsignal.SIGKILL: -9>, started='16:31:48') (30542) terminated with exit code Negsignal.SIGKILL
   [2021-06-19 16:32:54,403] {process_utils.py:66} INFO - Process psutil.Process(pid=30555, name='java', status='terminated', started='16:31:50') (30555) terminated with exit code None
   [2021-06-19 16:32:54,403] {process_utils.py:66} INFO - Process psutil.Process(pid=30544, name='airflow', status='terminated', started='16:31:48') (30544) terminated with exit code None
   [2021-06-19 16:32:54,404] {standard_task_runner.py:130} ERROR - Job 1354 was killed before it finished (likely due to running out of memory)
   ```
   
   The demo with hive operation runs success after deleting all the code of this PR. So please consider improving the PR or repeal 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.

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



[GitHub] [airflow] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > > The PR will cause dags that use hive operations to fail because they can't get the right PID.
   > > the hive task process info like these
   > > 1、airflow 20391 20170 11 17:41 ? 00:00:00 airflow task supervisor: ['airflow...
   > > 2、root 20394 20391 0 17:41 ? 00:00:00 sudo -E -H -u user airflow tasks run ...
   > > 3、user+ 20396 20394 61 17:41 ? 00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run...
   > > 4、user+ 20407 20396 99 17:41 ? 00:00:04 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4...
   > > The error log as follows:
   > > [2021-06-15 16:40:10,222] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   > > [2021-06-15 16:40:11,871] {local_task_job.py:193} WARNING - Recorded pid 30962 does not match the current pid 30960
   > > [2021-06-15 16:40:11,877] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30960
   > > [2021-06-15 16:40:11,901] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   > > [2021-06-15 16:40:11,902] {logging_mixin.py:104} INFO - Killing the Hive job
   > > The demo with hive operation runs success after deleting all the code of this PR. So please consider improving the PR or repeal it.
   > 
   > Interesting. cc: @ashb
   
   @ashb @jedcunningham hello, could you please help to take a look? thanks.


-- 
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] jedcunningham commented on a change in pull request #15537: Fix on_failure_callback when task receive SIGKILL

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



##########
File path: tests/jobs/test_local_task_job.py
##########
@@ -547,18 +550,83 @@ def task_function(ti):
         process = multiprocessing.Process(target=job1.run)
         process.start()
 
-        for _ in range(0, 10):
+        for _ in range(0, 20):
             ti.refresh_from_db()
-            if ti.state == State.RUNNING:
+            if ti.state == State.RUNNING and ti.pid is not None:
                 break
             time.sleep(0.2)
         assert ti.state == State.RUNNING
+        assert ti.pid is not None
         os.kill(ti.pid, signal.SIGTERM)
         process.join(timeout=10)
         assert failure_callback_called.value == 1
         assert task_terminated_externally.value == 1
         assert not process.is_alive()
 
+    def test_process_sigkill_call_on_failure_callback(self):
+        """
+        Test that ensures that when a task is killed with sigterm

Review comment:
       ```suggestion
           Test that ensures that when a task is killed with sigkill
   ```
   
   Should be sigkill, right?




-- 
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] huozhanfeng commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > > The PR will cause dags that use hive operations to fail because they can't get the right PID.
   > > the hive task process info like these
   > > 1、airflow 20391 20170 11 17:41 ? 00:00:00 airflow task supervisor: ['airflow...
   > > 2、root 20394 20391 0 17:41 ? 00:00:00 sudo -E -H -u user airflow tasks run ...
   > > 3、user+ 20396 20394 61 17:41 ? 00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run...
   > > 4、user+ 20407 20396 99 17:41 ? 00:00:04 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4...
   > > The error log as follows:
   > > [2021-06-15 16:40:10,222] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   > > [2021-06-15 16:40:11,871] {local_task_job.py:193} WARNING - Recorded pid 30962 does not match the current pid 30960
   > > [2021-06-15 16:40:11,877] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30960
   > > [2021-06-15 16:40:11,901] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   > > [2021-06-15 16:40:11,902] {logging_mixin.py:104} INFO - Killing the Hive job
   > > The demo with hive operation runs success after deleting all the code of this PR. So please consider improving the PR or repeal it.
   > 
   > Interesting. cc: @ashb
   
   @ashb hello, could you please help to take a look? thanks.


-- 
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] huozhanfeng commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   > Hi @huozhanfeng, I'm taking a look at fixing this but would like it if you can give me a reproducible step using Bash or python operator.
   
   Sure. In my opinion, you can mock a bash or python operator which contains a java sub-process to reproduce it. In my env, python operations that use the apache-sqoop tool fail to run, and hive operators fail to run.


-- 
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] uranusjr commented on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   Can `test_process_kill_call_on_failure_callback` and `test_process_sigkill_call_on_failure_callback` be combined with `pytest.mark.parametrize()`? They look much too similar to me. (Potentially `test_mark_success_on_success_callback` as well.)


-- 
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] huozhanfeng edited a comment on pull request #15537: Fix on_failure_callback when task receive SIGKILL

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


   The PR will cause dags that use hive operations to fail because they can't get the right PID.
   
   the hive task process info like these 
   1、airflow  20391 20170 11 17:41 ?        00:00:00 airflow task supervisor: ['airflow...
   2、root     20394 20391  0 17:41 ?        00:00:00 sudo -E -H -u user airflow tasks run ...
   3、user+ 20396 20394 61 17:41 ?        00:00:02 /usr/local/python3/bin/python3.7 /bin/airflow tasks run...
   4、user+ 20407 20396 99 17:41 ?        00:00:04 /usr/local/java/bin/java -Xmx1024m -Dhdp.version=2.6.5.0-292 -Djava.net.preferIPv4...
   
   The error log as follows:
   
   [2021-06-15 16:40:10,222] {hive.py:256} INFO - Logging initialized using configuration in file:/etc/hive/2.6.5.0-292/0/hive-log4j.properties
   [2021-06-15 16:40:11,871] {local_task_job.py:193} WARNING - Recorded pid 30962 does not match the current pid 30960
   [2021-06-15 16:40:11,877] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 30960
   [2021-06-15 16:40:11,901] {taskinstance.py:1264} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-06-15 16:40:11,902] {logging_mixin.py:104} INFO - Killing the Hive job
   
   The demo with hive operation runs success after deleting all the code of this PR. So please consider improving the PR or repeal 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.

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