hliu47 opened a new issue, #25297:
URL: https://github.com/apache/airflow/issues/25297

   ### Apache Airflow version
   
   2.2.5
   
   ### What happened
   
   `on_failure_callback` is not called when task is terminated externally.
   A similar issue was reported in 
[#14422](https://github.com/apache/airflow/issues/14422) and fixed in 
[#15172](https://github.com/apache/airflow/pull/15172).
   However, the code that fixed this was changed in a later PR 
[#16301](https://github.com/apache/airflow/pull/16301), after which 
`task_instance._run_finished_callback` is no longer called when SIGTERM is 
received
   
(https://github.com/apache/airflow/pull/16301/files#diff-d80fa918cc75c4d6aa582d5e29eeb812ba21371d6977fde45a4749668b79a515L85).
   
   ### What you think should happen instead
   
   `on_failure_callback` should be called when task fails regardless of how the 
task fails.
   
   ### How to reproduce
   
   DAG file:
   ```
   import datetime
   import pendulum
   from airflow.models import DAG
   from airflow.operators.bash_operator import BashOperator
    
    
   DEFAULT_ARGS = {
     'email': ['exam...@airflow.com']
   }
    
    
   TZ = pendulum.timezone("America/Los_Angeles")
    
   test_dag = DAG(
     dag_id='test_callback_in_manually_terminated_dag',
     schedule_interval='*/10 * * * *',
     default_args=DEFAULT_ARGS,
     catchup=False,
     start_date=datetime.datetime(2022, 7, 14, 0, 0, tzinfo=TZ)
   )
    
   with test_dag:
     BashOperator(
       task_id='manually_terminated_task',
       bash_command='echo start; sleep 60',
       on_failure_callback=lambda context: print('This on_failure_back should 
be called when task fails.')
     )
   ```
   While the task instance is running, either force quitting the scheduler or 
manually updating its state to None in the database will cause the task to get 
SIGTERM and terminate. In either case, a failure callback will not be called 
which does not match the behavior of previous versions of Airflow.
   
   The stack trace is attached below and `on_failure_callback` is not called.
   ```
   [2022-07-15, 02:02:24 UTC] {process_utils.py:124} INFO - Sending 
Signals.SIGTERM to group 10571. PIDs of all processes in the group: [10573, 
10575, 10571]
   [2022-07-15, 02:02:24 UTC] {process_utils.py:75} INFO - Sending the signal 
Signals.SIGTERM to group 10571
   [2022-07-15, 02:02:24 UTC] {taskinstance.py:1431} ERROR - Received SIGTERM. 
Terminating subprocesses.
   [2022-07-15, 02:02:24 UTC] {subprocess.py:99} INFO - Sending SIGTERM signal 
to process group
   [2022-07-15, 02:02:24 UTC] {process_utils.py:70} INFO - Process 
psutil.Process(pid=10575, status='terminated', started='02:02:11') (10575) 
terminated with exit code None
   [2022-07-15, 02:02:24 UTC] {taskinstance.py:1776} ERROR - Task failed with 
exception
   Traceback (most recent call last):
     File 
"/opt/python3.7/lib/python3.7/site-packages/airflow/operators/bash.py", line 
182, in execute
       cwd=self.cwd,
     File 
"/opt/python3.7/lib/python3.7/site-packages/airflow/hooks/subprocess.py", line 
87, in run_command
       for raw_line in iter(self.sub_process.stdout.readline, b''):
     File 
"/opt/python3.7/lib/python3.7/site-packages/airflow/models/taskinstance.py", 
line 1433, in signal_handler
       raise AirflowException("Task received SIGTERM signal")
   airflow.exceptions.AirflowException: Task received SIGTERM signal
   [2022-07-15, 02:02:24 UTC] {taskinstance.py:1289} INFO - Marking task as 
FAILED. dag_id=test_callback_in_manually_terminated_dag, 
task_id=manually_terminated_task, execution_date=20220715T015100, 
start_date=20220715T020211, end_date=20220715T020224
   [2022-07-15, 02:02:24 UTC] {logging_mixin.py:109} WARNING - 
/opt/python3.7/lib/python3.7/site-packages/airflow/utils/email.py:108 
PendingDeprecationWarning: Fetching SMTP credentials from configuration 
variables will be deprecated in a future release. Please set credentials using 
a connection instead.
   [2022-07-15, 02:02:24 UTC] {configuration.py:381} WARNING - section/key 
[smtp/smtp_user] not found in config
   [2022-07-15, 02:02:24 UTC] {email.py:214} INFO - Email alerting: attempt 1
   [2022-07-15, 02:02:24 UTC] {configuration.py:381} WARNING - section/key 
[smtp/smtp_user] not found in config
   [2022-07-15, 02:02:24 UTC] {email.py:214} INFO - Email alerting: attempt 1
   [2022-07-15, 02:02:24 UTC] {taskinstance.py:1827} ERROR - Failed to send 
email to: ['exam...@airflow.com']
   ...
   OSError: [Errno 101] Network is unreachable
   [2022-07-15, 02:02:24 UTC] {standard_task_runner.py:98} ERROR - Failed to 
execute job 159 for task manually_terminated_task (Task received SIGTERM 
signal; 10571)
   [2022-07-15, 02:02:24 UTC] {process_utils.py:70} INFO - Process 
psutil.Process(pid=10571, status='terminated', exitcode=1, started='02:02:11') 
(10571) terminated with exit code 1
   [2022-07-15, 02:02:24 UTC] {process_utils.py:70} INFO - Process 
psutil.Process(pid=10573, status='terminated', started='02:02:11') (10573) 
terminated with exit code None
   ```
   
   ### Operating System
   
   CentOS Linux 7
   
   ### Deployment
   
   Other Docker-based deployment
   
   ### Anything else
   
   This is an issue in 2.2.5. However, I notice that it appears to be fixed in 
the main branch by PR 
[#21877](https://github.com/apache/airflow/pull/21877/files#diff-62f7d8a52fefdb8e05d4f040c6d3459b4a56fe46976c24f68843dbaeb5a98487R1885-R1887)
 although it was not intended to fix this issue. Is there a timeline for 
getting that PR into a release? We are happy to test it out to see if it fixes 
the issue once it's released.
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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

Reply via email to