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/23 17:14:22 UTC

[GitHub] [airflow] madison-ookla opened a new issue #11086: on_failure_callback not called when task receives termination signal

madison-ookla opened a new issue #11086:
URL: https://github.com/apache/airflow/issues/11086


   **Apache Airflow version**: 1.10.7, 1.10.10, 1.10.12
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**: AWS EC2
   - **OS** (e.g. from /etc/os-release): Linux
   - **Kernel** (e.g. `uname -a`): Debian
   - **Install tools**:
   - **Others**:
   
   **What happened**:
   
   For the last several versions of Airflow, we've noticed that when a task receives a `SIGTERM` signal (currently represented as `Task exited with return code Negsignal.SIGKILL`, though previously represented as `Task exited with return code -9`), the failure email would be sent, but the `on_failure_callback` would not be called.
   
   This happened fairly frequently in the past for us as we had tasks that would consume high amounts of memory and occasionally we would have too many running on the same worker and the tasks would be OOM killed. In these instances, we would receive failure emails with the contents `detected as zombie` and the `on_failure_callback` would **not** be called. We were hoping #7025 would resolve this with the most recent upgrade (and we've also taken steps to reduce our memory footprint), but we just had this happen again recently. 
   
   **What you expected to happen**:
   
   If a tasks fails (even if the cause of the failure is a lack of resources), I would hope the `on_failure_callback` would still be called.
   
   **How to reproduce it**:
   
   Example DAG setup:
   
   <details><summary>CODE</summary>
   
   ```python
   # -*- coding: utf-8 -*-
   
   """
   # POC: On Failure Callback for SIGKILL
   """
   
   from datetime import datetime
   
   import numpy as np
   
   from airflow import DAG
   from airflow.api.common.experimental.trigger_dag import trigger_dag
   from airflow.operators.python_operator import PythonOperator
   
   
   def on_failure_callback(**context):
       print("===IN ON FAILURE CALLBACK===")
       print("Triggering another run of the task")
       trigger_dag("OOM_test_follower")
   
   
   def high_memory_task():
       l = []
       iteration = 0
       while True:
           print(f"Iteration: {iteration}")
           l.append(np.random.randint(1_000_000, size=(1000, 1000, 100)))
           iteration += 1
   
   
   def failure_task():
       raise ValueError("whoops")
   
   
   def print_context(**context):
       print("This DAG was launched by the failure callback")
       print(context)
   
   
   dag = DAG(
       dag_id="OOM_test",
       schedule_interval=None,
       catchup=False,
       default_args={
           "owner": "madison.bowden",
           "start_date": datetime(year=2019, month=7, day=1),
           "email": "your-email",
       },
   )
   
   with dag:
   
       PythonOperator(
           task_id="oom_task",
           python_callable=high_memory_task,
           on_failure_callback=on_failure_callback,
       )
   
   failure_dag = DAG(
       dag_id="Failure_test",
       schedule_interval=None,
       catchup=False,
       default_args={
           "owner": "madison.bowden",
           "start_date": datetime(year=2019, month=7, day=1),
           "email": "your-email",
       },
   )
   
   with failure_dag:
   
       PythonOperator(
           task_id="failure_task",
           python_callable=failure_task,
           on_failure_callback=on_failure_callback,
       )
   
   dag_follower = DAG(
       dag_id="OOM_test_follower",
       schedule_interval=None,
       catchup=False,
       default_args={
           "owner": "madison.bowden",
           "start_date": datetime(year=2019, month=7, day=1),
           "email": "your-email",
       },
   )
   
   with dag_follower:
   
       PythonOperator(
           task_id="oom_task_failure", python_callable=print_context, provide_context=True
       )
   
   ```
   
   </details>
   
   With the above example, the `Failure_test` should trigger a run of the `OOM_test_follower` DAG when it fails. The `OOM_test` DAG when triggered should quickly run out of memory and then **not** trigger a run of the `OOM_test_follower` DAG.
   
   
   **Anything else we need to know**:
   
   <!--
   
   How often does this problem occur? Once? Every time etc?
   
   Any relevant logs to include? Put them here in side a detail tag:
   <details><summary>x.log</summary> lots of stuff </details>
   
   -->
   


-- 
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] madison-ookla commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
madison-ookla commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-698572599


   @houqp Totally agree with your assessment, and if you can incorporate that in #10917 that would be amazing! We definitely only want the callback issued once, and ideally issued regardless what happens to the process (`SIGKILL` vs `SIGTERM`). FWIW we're using the CeleryExecutor.
   
   Thanks a ton!


----------------------------------------------------------------
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] turbaszek commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
turbaszek commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696900316


   Related to #10917 ?


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

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



[GitHub] [airflow] houqp commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-820826477


   @molejnik-mergebit this should have already been fixed in 2.0.1 through #10917. what version are you on?


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

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



[GitHub] [airflow] houqp commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-698114566


   yeah, those are two different issues. @madison-ookla what executor are you using?
   
   If you run into OOM, the raw task process will receive a SIGKILL instead of SIGTERM, which cannot be captured and handled by the process itself. #7025 doesn't solve this problem because it only handles cases where raw task process did not exit by itself or were killed by SIGKILL.
   
   I think to properly fix this bug, we will need to move the failure callback invocation into caller of raw task process, e.g. `local_task_job`. That way, we can just check for return code from the raw task and always invoke failure callback if it's not 0, which should cover the SIGKILL case.
   
   I will update my #10917 PR to cover 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.

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



[GitHub] [airflow] madison-ookla commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
madison-ookla commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696934180


   > Related to #10917 ?
   
   Good find! I think this issue is related, but not quite the same. This particular case is explicitly mentioned in @houqp's response [here](https://github.com/apache/airflow/pull/10917#issuecomment-694567832), and it looks like the change that would rectify this (moving callback handling to `local_scheduler_job` rather than `local_task_job`, I think) is out of scope for that particular 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.

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696890105


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


----------------------------------------------------------------
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] boring-cyborg[bot] commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696890105


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


----------------------------------------------------------------
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] turbaszek commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
turbaszek commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696900316


   Related to #10917 ?


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

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



[GitHub] [airflow] houqp edited a comment on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp edited a comment on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-825798723


   @molejnik-mergebit actually I take it back, this bug is not fixed, but it's now fixable after #10917, I believe @ephraimbuddy is working on 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] houqp commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-698114566


   yeah, those are two different issues. @madison-ookla what executor are you using?
   
   If you run into OOM, the raw task process will receive a SIGKILL instead of SIGTERM, which cannot be captured and handled by the process itself. #7025 doesn't solve this problem because it only handles cases where raw task process did not exit by itself or were killed by SIGKILL.
   
   I think to properly fix this bug, we will need to move the failure callback invocation into caller of raw task process, e.g. `local_task_job`. That way, we can just check for return code from the raw task and always invoke failure callback if it's not 0, which should cover the SIGKILL case.
   
   I will update my #10917 PR to cover 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.

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



[GitHub] [airflow] ashb closed issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
ashb closed issue #11086:
URL: https://github.com/apache/airflow/issues/11086


   


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

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



[GitHub] [airflow] houqp closed issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp closed issue #11086:
URL: https://github.com/apache/airflow/issues/11086


   


-- 
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] madison-ookla commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
madison-ookla commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-696934180


   > Related to #10917 ?
   
   Good find! I think this issue is related, but not quite the same. This particular case is explicitly mentioned in @houqp's response [here](https://github.com/apache/airflow/pull/10917#issuecomment-694567832), and it looks like the change that would rectify this (moving callback handling to `local_scheduler_job` rather than `local_task_job`, I think) is out of scope for that particular 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.

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



[GitHub] [airflow] molejnik-mergebit commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
molejnik-mergebit commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-817608794


   Hello, I have similar problem. Any ETA on fixing 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.

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



[GitHub] [airflow] madison-ookla commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
madison-ookla commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-698572599


   @houqp Totally agree with your assessment, and if you can incorporate that in #10917 that would be amazing! We definitely only want the callback issued once, and ideally issued regardless what happens to the process (`SIGKILL` vs `SIGTERM`). FWIW we're using the CeleryExecutor.
   
   Thanks a ton!


----------------------------------------------------------------
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] molejnik-mergebit commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
molejnik-mergebit commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-820932357


   @houqp thanks for the heads up, I'm still on old 1.10.12. Will try to update and check.


-- 
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] molejnik-mergebit commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
molejnik-mergebit commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-825885277


   Thanks for the update. I got it as well after update to 2.0.1, but I didn't have steps to reproduce 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] houqp commented on issue #11086: on_failure_callback not called when task receives termination signal

Posted by GitBox <gi...@apache.org>.
houqp commented on issue #11086:
URL: https://github.com/apache/airflow/issues/11086#issuecomment-825798723


   @molejnik-mergebit actually I take it back, this bug is not fixed, but it's not fixable after #10917, I believe @ephraimbuddy is working on 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