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/03/08 22:23:11 UTC

[GitHub] [airflow] saurabhladhe opened a new issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

saurabhladhe opened a new issue #14672:
URL: https://github.com/apache/airflow/issues/14672


   
   **Apache Airflow version**:2.0.1
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):1.18.10
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**:Azure Kubernetes Service 
   - **OS** (e.g. from /etc/os-release):Debian GNU/Linux 10
   - **Kernel** (e.g. `uname -a`):Linux airflow-k8s-scheduler-blabla-xvv5f 5.4.0-1039-azure #41~18.04.1-Ubuntu SMP Mon Jan 18 14:00:01 UTC 2021 x86_64 GNU/Linux
   - **Install tools**:
   - **Others**:
   
   **What happened**: 
   tasks keep failing with sigterm
   <details>
   <summary>Task fails due to sigterm before it can finish- </summary>
   [2021-03-08 20:39:14,503] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]> from DB
   [2021-03-08 20:39:14,878] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]>
   [2021-03-08 20:39:14,880] {base_job.py:219} DEBUG - [heartbeat]
   [2021-03-08 20:39:21,241] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]> from DB
   [2021-03-08 20:39:21,670] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]>
   [2021-03-08 20:39:21,672] {base_job.py:219} DEBUG - [heartbeat]
   [2021-03-08 20:39:27,736] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]> from DB
   [2021-03-08 20:39:28,360] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [None]>
   [2021-03-08 20:39:28,362] {local_task_job.py:188} WARNING - State of this instance has been externally set to None. Terminating instance.
   [2021-03-08 20:39:28,363] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 21
   [2021-03-08 20:39:50,119] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-08 20:39:50,119] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [running]> from DB
   [2021-03-08 20:39:51,328] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xxxxx_xxxxxx_xxx 2021-03-08T19:20:00+00:00 [queued]>
   [2021-03-08 20:39:51,329] {taskinstance.py:1455} ERROR - Task received SIGTERM signal
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1112, in _run_raw_task
       self._prepare_and_execute_task_with_callbacks(context, task)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1285, in _prepare_and_execute_task_with_callbacks
       result = self._execute_task(context, task_copy)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1315, in _execute_task
       result = task_copy.execute(context=context)
     File "/opt/airflow/mnt/dags/xyz/operators/xyz_data_sync_operator.py", line 73, in execute
       XyzUpsertClass(self.dest_conn_id, self.table, self.schema, src_res, drop_missing_columns=self.drop_missing_columns).execute()
     File "/opt/airflow/mnt/dags/xyz/operators/xyz_data_sync_operator.py", line 271, in execute
       cursor.execute( f"UPDATE {self.schema}.{self.table} SET {up_template} WHERE {pk_template}",row)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1241, in signal_handler
       raise AirflowException("Task received SIGTERM signal")
   airflow.exceptions.AirflowException: Task received SIGTERM signal
   [2021-03-08 20:39:51,409] {taskinstance.py:1862} DEBUG - Task Duration set to 1055.564798
   [2021-03-08 20:39:51,410] {taskinstance.py:1503} INFO - Marking task as FAILED. dag_id=xyz_sync task_id=xxxxx, execution_date=20210308T192000, start_date=20210308T202215, end_date=20210308T203951
   </details>
   <details>
   <summary>This also happens with a successful task-</summary>
   [2021-03-08 21:46:01,466] {__init__.py:124} DEBUG - Preparing lineage inlets and outlets
   [2021-03-08 21:46:01,466] {__init__.py:168} DEBUG - inlets: [], outlets: []
   [2021-03-08 21:46:01,467] {logging_mixin.py:104} INFO - {'conf': <airflow.configuration.AirflowConfigParser object at 0x7fc0f7f2a910>, 'dag': <DAG: canary_dag>, 'dag_run': <DagRun canary_dag @ 2021-03-08 20:10:00+00:00: scheduled__2021-03-08T20:10:00+00:00, externally triggered: False>, 'ds_nodash': '20210308', 'execution_date': DateTime(2021, 3, 8, 20, 10, 0, tzinfo=Timezone('+00:00')), 'inlets': [], 'macros': <module 'airflow.macros' from '/home/airflow/.local/lib/python3.7/site-packages/airflow/macros/__init__.py'>, 'next_ds': '2021-03-08', 'next_ds_nodash': '20210308', 'next_execution_date': DateTime(2021, 3, 8, 21, 10, 0, tzinfo=Timezone('UTC')), 'outlets': [], 'params': {}, 'prev_ds': '2021-03-08', 'prev_ds_nodash': '20210308', 'prev_execution_date': DateTime(2021, 3, 8, 19, 10, 0, tzinfo=Timezone('UTC')), 'prev_execution_date_success': <Proxy at 0x7fc0e3d0d780 with factory <function TaskInstance.get_template_context.<locals>.<lambda> at 0x7fc0e3d2ad40>>, 'prev_start_date_su
 ccess': <Proxy at 0x7fc0e3d0d7d0 with factory <function TaskInstance.get_template_context.<locals>.<lambda> at 0x7fc0e3ff8830>>, 'run_id': 'scheduled__2021-03-08T20:10:00+00:00', 'task': <Task(PythonOperator): print_the_context>, 'task_instance': <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [running]>, 'task_instance_key_str': 'canary_dag__print_the_context__20210308', 'test_mode': False, 'ti': <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [running]>, 'tomorrow_ds': '2021-03-09', 'tomorrow_ds_nodash': '20210309', 'ts': '2021-03-08T20:10:00+00:00', 'ts_nodash': '20210308T201000', 'ts_nodash_with_tz': '20210308T201000+0000', 'var': {'json': None, 'value': None}, 'yesterday_ds': '2021-03-07', 'yesterday_ds_nodash': '20210307', 'templates_dict': None}
   [2021-03-08 21:46:01,467] {logging_mixin.py:104} INFO - 2021-03-08
   [2021-03-08 21:46:01,467] {python.py:118} INFO - Done. Returned value was: Whatever you return gets printed in the logs
   [2021-03-08 21:46:21,690] {__init__.py:88} DEBUG - Lineage called with inlets: [], outlets: []
   [2021-03-08 21:46:21,691] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [running]> from DB
   [2021-03-08 21:46:32,042] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [running]>
   [2021-03-08 21:46:32,051] {taskinstance.py:1166} INFO - Marking task as SUCCESS. dag_id=canary_dag, task_id=print_the_context, execution_date=20210308T201000, start_date=20210308T214431, end_date=20210308T214632
   [2021-03-08 21:46:32,051] {taskinstance.py:1862} DEBUG - Task Duration set to 120.361368
   [2021-03-08 21:46:38,577] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [running]> from DB
   [2021-03-08 21:46:51,150] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: canary_dag.print_the_context 2021-03-08T20:10:00+00:00 [success]>
   [2021-03-08 21:46:51,152] {local_task_job.py:188} WARNING - State of this instance has been externally set to success. Terminating instance.
   [2021-03-08 21:46:51,153] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 22
   [2021-03-08 21:46:59,668] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-08 21:46:59,669] {cli_action_loggers.py:84} DEBUG - Calling callbacks: []
   [2021-03-08 21:46:59,757] {process_utils.py:66} INFO - Process psutil.Process(pid=22, status='terminated', exitcode=1, started='21:44:41') (22) terminated with exit code 1
   [2021-03-08 21:46:59,758] {base_job.py:219} DEBUG - [heartbeat]
   </details>
   
   **What you expected to happen**:
   Tasks marked correctly instead of receiving sigterm
   
   **How to reproduce it**:
   mostly occurs when a large number of dags (100+) are running on AKS
   
   **Anything else we need to know**:
   DAGs that run longer or have more than 10-12 tasks (that may run long) seem to have a higher probability of this happening to them
   


----------------------------------------------------------------
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] lidalei commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   > @lidalei : awesome, thanks for confirming, do let me know if it does show up again, we will probably try this setting on our next deploy as well
   
   I can confirm it solved our issues.


-- 
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] saurabhladhe edited a comment on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @jedcunningham : 
   I checked multiple instances and not seeing any evicted pods for the ones that got sigterm 
   
   K8s events: 
   16m         Normal    Scheduled                      pod/xyzypod.somestr        Successfully assigned <namesapce>/xyz.a205feb0f53c40abab3dddc46939ce38 to aks-<namesapce>
   16m         Normal    Pulled                         pod/xyzypod.somestr        Container image "airflow:2.0.1" already present on machine
   16m         Normal    Created                        pod/xyzypod.somestr        Created container base
   16m         Normal    Started                        pod/xyzypod.somestr        Started container base
   
   
   and got sigterm in the logs with some traceback related to sqlalchemy:
   
   `[2021-03-12 18:46:30,373] {xyz.py:104} DEBUG - Processing finished for pipeline id: 570 (0:00:07.470379 s elapsed)
   [2021-03-12 18:46:30,373] {xyz.py:91} DEBUG - Processing pipeline id: 316
   [2021-03-12 18:46:34,417] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: somedagid.sometaskid 2021-03-01T01:14:00+00:00 [failed]>
   [2021-03-12 18:46:34,419] {local_task_job.py:188} WARNING - State of this instance has been externally set to failed. Terminating instance.
   [2021-03-12 18:46:34,420] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 22
   [2021-03-12 18:46:35,793] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-12 18:46:35,794] {xyz.py:100} ERROR - Processing failed for pipeline id: 316 -- Reason: Task received SIGTERM signal
   Traceback (most recent call last):
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 96, in consume_context
       status_thresholds, context)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 117, in _process_pipeline_id
       count = _count_available_records(read_hook, count_sql, pipeline_id)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 233, in _count_available_records
       df = hook.get_pandas_df(sql, parameters={'pipeline_id': pipeline_id})
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/dbapi.py", line 116, in get_pandas_df
       with closing(self.get_conn()) as conn:
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/providers/postgres/hooks/postgres.py", line 83, in get_conn
       conn = self.connection or self.get_connection(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/base.py", line 63, in get_connection
       conn = Connection.get_connection_from_secrets(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/connection.py", line 351, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/utils/session.py", line 65, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/secrets/metastore.py", line 37, in get_connection
       conn = session.query(Connection).filter(Connection.conn_id == conn_id).first()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3429, in first
       ret = list(self[0:1])
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3203, in __getitem__
       return list(res)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3535, in __iter__
       return self._execute_and_instances(context)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3557, in _execute_and_instances
       querycontext, self._connection_from_session, close_with_result=True
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3572, in _get_bind_args
       mapper=self._bind_mapper(), clause=querycontext.statement, **kw
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3550, in _connection_from_session
       conn = self.session.connection(**kw)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1145, in connection
       execution_options=execution_options,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1151, in _connection_for_bind
       engine, execution_options
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 433, in _connection_for_bind
       conn = bind._contextual_connect()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2302, in _contextual_connect
       self._wrap_pool_connect(self.pool.connect, None),
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2336, in _wrap_pool_connect
       return fn()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 364, in connect
       return _ConnectionFairy._checkout(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 778, in _checkout
       fairy = _ConnectionRecord.checkout(pool)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 495, in checkout
       rec = pool._do_get()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/impl.py", line 241, in _do_get
       return self._create_connection()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 309, in _create_connection
       return _ConnectionRecord(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 440, in __init__
       self.__connect(first_connect_check=True)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 661, in __connect
       pool.logger.debug("Error on connect(): %s", e)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/langhelpers.py", line 70, in __exit__
       with_traceback=exc_tb,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 182, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 656, in __connect
       connection = pool._invoke_creator(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/strategies.py", line 114, in connect
       return dialect.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 508, in connect
       return self.dbapi.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/psycopg2/__init__.py", line 127, in connect
       conn = _connect(dsn, connection_factory=connection_factory, **kwasync)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1241, in signal_handler
       raise AirflowException("Task received SIGTERM signal")
   airflow.exceptions.AirflowException: Task received SIGTERM signal`
   
   


----------------------------------------------------------------
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 issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @stroykova can you create a separate issue for the behaviour you experienced. It looks different from other issues here because you're using multiprocessing in your code


-- 
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] Nimesh-K-Makwana commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

Posted by GitBox <gi...@apache.org>.
Nimesh-K-Makwana commented on issue #14672:
URL: https://github.com/apache/airflow/issues/14672#issuecomment-914192397


   > Hello,
   > 
   > I am facing the same issue:
   > airflow 2.1.3 (tested also with 2.1.2)
   > backend: postgresql
   > executor: LocalExecutor
   > 
   > I have modified the variables killed_task_cleanup_time and schedule_after_task_execution to resp. 100000 and False.
   > I have also installed airflow as non root user and set the default run_as_user to be airflow.
   > 
   > My tasks are getting constantly killed in backfill mode with the traceback:
   > 
   > `[2021-09-07 10:21:20,185] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 73011`
   > 
   > Honestly, I am a bit discouraged at this point, could you help me please ? tks
   
   Can u continue the thread here: [18041](https://github.com/apache/airflow/issues/18041)
   Also provide task and scheduler logs.


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

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

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



[GitHub] [airflow] jedcunningham commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   I believe this is fixed with #16289.


-- 
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 #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   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] Nimesh-K-Makwana commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

Posted by GitBox <gi...@apache.org>.
Nimesh-K-Makwana commented on issue #14672:
URL: https://github.com/apache/airflow/issues/14672#issuecomment-912957655


   I am getting this issue in 2.1.3 as well. All retires are also getting sigkilled. 
   Error : airflow.exceptions.AirflowException: Task received SIGTERM signal


-- 
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] Nimesh-K-Makwana edited a comment on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

Posted by GitBox <gi...@apache.org>.
Nimesh-K-Makwana edited a comment on issue #14672:
URL: https://github.com/apache/airflow/issues/14672#issuecomment-912957655






-- 
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @jedcunningham : 
   I checked multiple instances and not seeing any evicted pods for the ones that got sigterm 
   `16m         Normal    Scheduled                      pod/xyzypod.somestr        Successfully assigned <namesapce>/xyz.a205feb0f53c40abab3dddc46939ce38 to aks-<namesapce>
   16m         Normal    Pulled                         pod/xyzypod.somestr        Container image "airflow:2.0.1" already present on machine
   16m         Normal    Created                        pod/xyzypod.somestr        Created container base
   16m         Normal    Started                        pod/xyzypod.somestr        Started container base`
   and got sigterm in the logs with some traceback related to sqlalchemy:
   `[2021-03-12 18:46:30,373] {xyz.py:104} DEBUG - Processing finished for pipeline id: 570 (0:00:07.470379 s elapsed)
   [2021-03-12 18:46:30,373] {xyz.py:91} DEBUG - Processing pipeline id: 316
   [2021-03-12 18:46:34,417] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: somedagid.sometaskid 2021-03-01T01:14:00+00:00 [failed]>
   [2021-03-12 18:46:34,419] {local_task_job.py:188} WARNING - State of this instance has been externally set to failed. Terminating instance.
   [2021-03-12 18:46:34,420] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 22
   [2021-03-12 18:46:35,793] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-12 18:46:35,794] {xyz.py:100} ERROR - Processing failed for pipeline id: 316 -- Reason: Task received SIGTERM signal
   Traceback (most recent call last):
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 96, in consume_context
       status_thresholds, context)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 117, in _process_pipeline_id
       count = _count_available_records(read_hook, count_sql, pipeline_id)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 233, in _count_available_records
       df = hook.get_pandas_df(sql, parameters={'pipeline_id': pipeline_id})
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/dbapi.py", line 116, in get_pandas_df
       with closing(self.get_conn()) as conn:
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/providers/postgres/hooks/postgres.py", line 83, in get_conn
       conn = self.connection or self.get_connection(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/base.py", line 63, in get_connection
       conn = Connection.get_connection_from_secrets(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/connection.py", line 351, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/utils/session.py", line 65, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/secrets/metastore.py", line 37, in get_connection
       conn = session.query(Connection).filter(Connection.conn_id == conn_id).first()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3429, in first
       ret = list(self[0:1])
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3203, in __getitem__
       return list(res)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3535, in __iter__
       return self._execute_and_instances(context)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3557, in _execute_and_instances
       querycontext, self._connection_from_session, close_with_result=True
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3572, in _get_bind_args
       mapper=self._bind_mapper(), clause=querycontext.statement, **kw
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3550, in _connection_from_session
       conn = self.session.connection(**kw)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1145, in connection
       execution_options=execution_options,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1151, in _connection_for_bind
       engine, execution_options
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 433, in _connection_for_bind
       conn = bind._contextual_connect()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2302, in _contextual_connect
       self._wrap_pool_connect(self.pool.connect, None),
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2336, in _wrap_pool_connect
       return fn()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 364, in connect
       return _ConnectionFairy._checkout(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 778, in _checkout
       fairy = _ConnectionRecord.checkout(pool)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 495, in checkout
       rec = pool._do_get()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/impl.py", line 241, in _do_get
       return self._create_connection()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 309, in _create_connection
       return _ConnectionRecord(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 440, in __init__
       self.__connect(first_connect_check=True)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 661, in __connect
       pool.logger.debug("Error on connect(): %s", e)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/langhelpers.py", line 70, in __exit__
       with_traceback=exc_tb,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 182, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 656, in __connect
       connection = pool._invoke_creator(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/strategies.py", line 114, in connect
       return dialect.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 508, in connect
       return self.dbapi.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/psycopg2/__init__.py", line 127, in connect
       conn = _connect(dsn, connection_factory=connection_factory, **kwasync)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1241, in signal_handler
       raise AirflowException("Task received SIGTERM signal")
   airflow.exceptions.AirflowException: Task received SIGTERM signal`
   
   


----------------------------------------------------------------
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] sandeepsaimone edited a comment on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Hi Guys,
   
   I am running Mwaa version 2.0.2 and the dag run more than 5hrs, but in the end the task are marked has failed and there is no error in the logs. Below are the parameters update under airflow config at mwaa environment.
   
   Note :- if below parameters has been setup then i am ending up SIGTERM error.
   
   airflow.airflow_core_killed_task_cleanup_time=604800
   celery.worker_autoscale=10,2
   scheduler.scheduler_health_check_threshold=120
   scheduler.scheduler_heartbeat_sec=60 
   
   Can any one help us out.


-- 
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] sandeepsaimone commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Hi Guys,
   
   I am running Mwaa version 2.0.2 and the dag run more than 5hrs, but in the end the task are marked has failed and there is no error in the logs. Below are the parameters update under airflow config at mwaa environment.
   
   airflow.airflow_core_killed_task_cleanup_time=604800
   celery.worker_autoscale=10,2
   scheduler.scheduler_health_check_threshold=120
   scheduler.scheduler_heartbeat_sec=60 
   
   Can any one help us out.


-- 
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Found an old post similar to this (https://stackoverflow.com/questions/54313601/airflow-signals-sigterm-to-subprocesses-unexpectedly/63175235#63175235), will try changing killed_task_cleanup_time config param to see if it helps


----------------------------------------------------------------
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @lidalei : awesome, thanks for confirming, do let me know if it does show up again, we will probably try this setting on our next deploy 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] lidalei commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   tl;dr: set [schedule_after_task_execution](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#schedule-after-task-execution) to false by either updating your airflow.cfg (recommended) or setting AIRFLOW__SCHEDULER__SCHEDULE_AFTER_TASK_EXECUTION to False.
   
   
   We had the same issue and with the help of sentry to look through the whole stack trace, I found out why.
   
   The buggy code block is [taskinstance.py#L1182-L1201](https://github.com/apache/airflow/blob/master/airflow/models/taskinstance.py#L1182-L1201):
   
   ```
           # Recording SUCCESS
           self.end_date = timezone.utcnow()
           self.log.info(
               'Marking task as SUCCESS. '
               'dag_id=%s, task_id=%s, execution_date=%s, start_date=%s, end_date=%s',
               self.dag_id,
               self.task_id,
               self._date_or_empty('execution_date'),
               self._date_or_empty('start_date'),
               self._date_or_empty('end_date'),
           )
           self.set_duration()
           if not test_mode:
               session.add(Log(self.state, self))
               session.merge(self)
   
           session.commit()
   
           if not test_mode:
               self._run_mini_scheduler_on_child_tasks(session)
   ```
   
   Looking at the code: after marking a task SUCCESS and **commit**, if it is not test mode, it will call a potentially expensive function _run_mini_scheduler_on_child_tasks. And [local_task_job.py#L179-L199](https://github.com/apache/airflow/blob/master/airflow/jobs/local_task_job.py#L179-L199) will detect the task SUCCESS very soon and since the task is not running, it will terminate the process which might be still executing _run_mini_scheduler_on_child_tasks:
   
   ```
           if ti.state == State.RUNNING:
               ...
           elif self.task_runner.return_code() is None and hasattr(self.task_runner, 'process'):
               self.log.warning(
                   "State of this instance has been externally set to %s. " "Terminating instance.", ti.state
               )
               self.task_runner.terminate()
   ```
   
   This is proven by the log @saurabhladhe shared (the line numbers diverge a bit because the log was logged by Airflow 2.0.1):
   ```
   [2021-03-08 21:46:32,051] {taskinstance.py:1166} INFO - Marking task as SUCCESS. dag_id=canary_dag, task_id=print_the_context, execution_date=20210308T201000, start_date=20210308T214431, end_date=20210308T214632
   
   ...
   
   [2021-03-08 21:46:51,152] {local_task_job.py:188} WARNING - State of this instance has been externally set to success. Terminating instance.
   
   [2021-03-08 21:46:51,153] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 22
   ```
   
   So the mitigation is to make _run_mini_scheduler_on_child_tasks cheap, which is an optimization controlled by [schedule_after_task_execution](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#schedule-after-task-execution) and can be disabled.
   
   ```
       def _run_mini_scheduler_on_child_tasks(self, session=None) -> None:
           if conf.getboolean('scheduler', 'schedule_after_task_execution', fallback=True):
   ```
   
   The proper fix would start from changing
   ```
           if not test_mode:
               session.add(Log(self.state, self))
               session.merge(self)
   
           session.commit()
   
           if not test_mode:
               self._run_mini_scheduler_on_child_tasks(session)
   ```
   
   to 
   
   ```
           if not test_mode:
               session.add(Log(self.state, self))
               session.merge(self)
               self._run_mini_scheduler_on_child_tasks(session)
   
           session.commit()
   ```
   
   However, _run_mini_scheduler_on_child_tasks might got an OperationalError and roll back the session completely [taskinstance.py#L1248](https://github.com/apache/airflow/blob/858f93cb79384c1034f3fa081bb0cc246582db94/airflow/models/taskinstance.py#L1248). And we will falsely mark the task failure, though it is only the optimization failure. So I'd leave it to someone who knows the code better to fix it properly. Personally I'd suggest to remove this optimization completely.


-- 
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   I observed that this same issue showed up on another system at my company which was running astronomer on EKS. While redeploying, we got " ERROR - Received SIGTERM. Terminating subprocesses." in the task that was in the running state before the deployment started.
   Based on this, we thought the issue might also be related to liveness probe failure which causes the scheduler to restart frequently 
   (similar to https://github.com/apache/airflow/issues/14261) 
   however, when we disabled the liveness probe, we were still seeing the sigterm issue popping up in large dags
   
   


----------------------------------------------------------------
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @lidalei : thanks for looking into it and sharing this
   btw, did setting AIRFLOW__SCHEDULER__SCHEDULE_AFTER_TASK_EXECUTION to False help resolve this issue in your case?


-- 
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] saurabhladhe edited a comment on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @jedcunningham : 
   I checked multiple instances and not seeing any evicted pods for the ones that got sigterm 
   16m         Normal    Scheduled                      pod/xyzypod.somestr        Successfully assigned <namesapce>/xyz.a205feb0f53c40abab3dddc46939ce38 to aks-<namesapce>
   16m         Normal    Pulled                         pod/xyzypod.somestr        Container image "airflow:2.0.1" already present on machine
   16m         Normal    Created                        pod/xyzypod.somestr        Created container base
   16m         Normal    Started                        pod/xyzypod.somestr        Started container base
   and got sigterm in the logs with some traceback related to sqlalchemy:
   
   `[2021-03-12 18:46:30,373] {xyz.py:104} DEBUG - Processing finished for pipeline id: 570 (0:00:07.470379 s elapsed)
   [2021-03-12 18:46:30,373] {xyz.py:91} DEBUG - Processing pipeline id: 316
   [2021-03-12 18:46:34,417] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: somedagid.sometaskid 2021-03-01T01:14:00+00:00 [failed]>
   [2021-03-12 18:46:34,419] {local_task_job.py:188} WARNING - State of this instance has been externally set to failed. Terminating instance.
   [2021-03-12 18:46:34,420] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 22
   [2021-03-12 18:46:35,793] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-12 18:46:35,794] {xyz.py:100} ERROR - Processing failed for pipeline id: 316 -- Reason: Task received SIGTERM signal
   Traceback (most recent call last):
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 96, in consume_context
       status_thresholds, context)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 117, in _process_pipeline_id
       count = _count_available_records(read_hook, count_sql, pipeline_id)
     File "/opt/airflow/mnt/dags/gas/pentaho/tasks/xyz.py", line 233, in _count_available_records
       df = hook.get_pandas_df(sql, parameters={'pipeline_id': pipeline_id})
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/dbapi.py", line 116, in get_pandas_df
       with closing(self.get_conn()) as conn:
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/providers/postgres/hooks/postgres.py", line 83, in get_conn
       conn = self.connection or self.get_connection(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/hooks/base.py", line 63, in get_connection
       conn = Connection.get_connection_from_secrets(conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/connection.py", line 351, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/utils/session.py", line 65, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/secrets/metastore.py", line 37, in get_connection
       conn = session.query(Connection).filter(Connection.conn_id == conn_id).first()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3429, in first
       ret = list(self[0:1])
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3203, in __getitem__
       return list(res)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3535, in __iter__
       return self._execute_and_instances(context)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3557, in _execute_and_instances
       querycontext, self._connection_from_session, close_with_result=True
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3572, in _get_bind_args
       mapper=self._bind_mapper(), clause=querycontext.statement, **kw
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/query.py", line 3550, in _connection_from_session
       conn = self.session.connection(**kw)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1145, in connection
       execution_options=execution_options,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 1151, in _connection_for_bind
       engine, execution_options
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/orm/session.py", line 433, in _connection_for_bind
       conn = bind._contextual_connect()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2302, in _contextual_connect
       self._wrap_pool_connect(self.pool.connect, None),
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 2336, in _wrap_pool_connect
       return fn()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 364, in connect
       return _ConnectionFairy._checkout(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 778, in _checkout
       fairy = _ConnectionRecord.checkout(pool)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 495, in checkout
       rec = pool._do_get()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/impl.py", line 241, in _do_get
       return self._create_connection()
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 309, in _create_connection
       return _ConnectionRecord(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 440, in __init__
       self.__connect(first_connect_check=True)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 661, in __connect
       pool.logger.debug("Error on connect(): %s", e)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/langhelpers.py", line 70, in __exit__
       with_traceback=exc_tb,
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/util/compat.py", line 182, in raise_
       raise exception
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/pool/base.py", line 656, in __connect
       connection = pool._invoke_creator(self)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/strategies.py", line 114, in connect
       return dialect.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 508, in connect
       return self.dbapi.connect(*cargs, **cparams)
     File "/home/airflow/.local/lib/python3.7/site-packages/psycopg2/__init__.py", line 127, in connect
       conn = _connect(dsn, connection_factory=connection_factory, **kwasync)
     File "/home/airflow/.local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 1241, in signal_handler
       raise AirflowException("Task received SIGTERM signal")
   airflow.exceptions.AirflowException: Task received SIGTERM signal`
   
   


----------------------------------------------------------------
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] saurabhladhe commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Got the same for a successful task as well:
   events:
   17m         Normal    Scheduled                      pod/xyztaskinstance.somestr         Successfully assigned <namespace>/xyztask.somestr
   17m         Normal    Pulled                         pod/xyztaskinstance.somestr         Container image "airflow:2.0.1" already present on machine
   17m         Normal    Created                        pod/xyztaskinstance.somestr         Created container base
   17m         Normal    Started                        pod/xyztaskinstance.somestr         Started container base
   logs:
   `[2021-03-12 19:40:30,924] {natgas_update_pipeline_aggregate_group2.py:104} DEBUG - Processing finished for pipeline id: 500 (0:00:09.669772 s elapsed)
   [2021-03-12 19:40:30,924] {natgas_update_pipeline_aggregate_group2.py:108} INFO - Processed 12 pipeline ids (12 successfully)(0:04:25.557094 s elapsed)
   [2021-03-12 19:40:30,925] {python.py:118} INFO - Done. Returned value was: None
   [2021-03-12 19:40:30,925] {__init__.py:88} DEBUG - Lineage called with inlets: [], outlets: []
   [2021-03-12 19:40:30,925] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xyztaskinstance 2021-03-01T01:28:00+00:00 [running]> from DB
   [2021-03-12 19:41:09,777] {taskinstance.py:570} DEBUG - Refreshing TaskInstance <TaskInstance: xyztaskinstance 2021-03-01T01:28:00+00:00 [running]> from DB
   [2021-03-12 19:41:09,800] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xyztaskinstance 2021-03-01T01:28:00+00:00 [failed]>
   [2021-03-12 19:41:09,828] {taskinstance.py:1166} INFO - Marking task as SUCCESS. dag_id=xyz, task_id=xyz, execution_date=20210301T012800, start_date=20210312T193433, end_date=20210312T194109
   [2021-03-12 19:41:09,828] {taskinstance.py:1862} DEBUG - Task Duration set to 396.729006
   [2021-03-12 19:41:10,970] {taskinstance.py:605} DEBUG - Refreshed TaskInstance <TaskInstance: xyztaskinstance 2021-03-01T01:28:00+00:00 [failed]>
   [2021-03-12 19:41:10,971] {local_task_job.py:188} WARNING - State of this instance has been externally set to failed. Terminating instance.
   [2021-03-12 19:41:10,972] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 21
   [2021-03-12 19:41:14,128] {taskinstance.py:1239} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-03-12 19:41:14,129] {cli_action_loggers.py:84} DEBUG - Calling callbacks: []
   [2021-03-12 19:41:14,150] {process_utils.py:66} INFO - Process psutil.Process(pid=21, status='terminated', exitcode=1, started='19:34:41') (21) terminated with exit code 1
   [2021-03-12 19:41:14,151] {base_job.py:219} DEBUG - [heartbeat]`
   


----------------------------------------------------------------
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] laserpedro commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Hello, 
   
   I am facing the same issue:
   airflow 2.1.3 (tested also with 2.1.2)
   backend: postgresql
   executor: LocalExecutor
   
   I have modified the variables killed_task_cleanup_time and schedule_after_task_execution to resp. 100000 and False.
   I have also installed airflow as non root user and set the default run_as_user to be airflow. 
   
   My tasks are getting constantly killed in backfill mode with the traceback:
   
   `[2021-09-07 10:21:20,185] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 73011`
   
   Honestly, I am a bit discouraged at this point, could you help me please ? tks


-- 
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] sandeepsaimone removed a comment on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

Posted by GitBox <gi...@apache.org>.
sandeepsaimone removed a comment on issue #14672:
URL: https://github.com/apache/airflow/issues/14672#issuecomment-1013568265


   Hi Guys,
   
   I am running Mwaa version 2.0.2 and the dag run more than 5hrs, but in the end the task are marked has failed and there is no error in the logs. Below are the parameters update under airflow config at mwaa environment.
   
   Note :- if below parameters has been setup then i am ending up SIGTERM error.
   
   airflow.airflow_core_killed_task_cleanup_time=604800
   celery.worker_autoscale=10,2
   scheduler.scheduler_health_check_threshold=120
   scheduler.scheduler_heartbeat_sec=60 
   
   Can any one help us out.


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

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

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



[GitHub] [airflow] jedcunningham commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   @saurabhladhe, can you try to grab the events for one of the "sigterm before it finished" pods and see if it was evicted? 


----------------------------------------------------------------
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] jianqiuhuang commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   I'm having a similar issue. [Tasks marked as success externally](https://github.com/apache/airflow/blob/9c8391a13f6ba29749675cf23f2f874f96b0cc8c/airflow/jobs/local_task_job.py#L197) and then `SIGTERM` is send to the process immediately after. 
   
   I think it's because the timeout is defaulted to `0` https://github.com/apache/airflow/blob/9c8391a13f6ba29749675cf23f2f874f96b0cc8c/airflow/task/task_runner/standard_task_runner.py#L101


-- 
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 closed issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   


-- 
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] lidalei commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   > @lidalei : thanks for looking into it and sharing this
   > 
   > btw, did setting AIRFLOW__SCHEDULER__SCHEDULE_AFTER_TASK_EXECUTION to False help resolve this issue in your case?
   
   No problem. It probably solved. We had constantly received the alerts before setting it to False. It has been more than 6 hours since we set it to False, and we don't see it any more.


-- 
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] jtommi commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   If it can help, for me it also solved the issue.
   I'm using a simple local executor with 2 docker containers (scheduler and webserver).
   The root cause is probably a rather slow Python script using pandas.


-- 
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] aprettyloner commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Also seeing this on Airflow 2.2.3


-- 
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] denysivanov commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   Hello, 
   
   v2.1.3 - having the same issue 
   
   [2021-09-20 14:15:34,615] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 8709
   [2021-09-20 14:15:34,839] {taskinstance.py:1235} ERROR - Received SIGTERM. Terminating subprocesses.


-- 
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] stroykova commented on issue #14672: Tasks intermittently get terminated with SIGTERM on K8s executor

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


   I use airflow in a data science project with lots of multiprocessing.
   I also have the same issue with SIGTERM signal. Here is a code example that shows using multiprocessing produces sigterm:
   
   ```
   from pebble import ProcessPool
   import multiprocessing 
   import signal 
   
   def function(foo, bar=0):
       print(multiprocessing.current_process().pid)
   
   def task_done(future):
       future.result()  # blocks until results are ready
   
   def process():
       def signal_handler(signum, frame): 
           raise ValueError("received SIGTERM signal") 
       signal.signal(signal.SIGTERM, signal_handler)
       with ProcessPool(max_workers=5, max_tasks=10) as pool:
           for i in range(0, 10):
               future = pool.schedule(function, args=[i], timeout=10)
               future.add_done_callback(task_done)
   ```
   
   I added sigterm handler and see it in every spawned process. I do not completely understand why this happens.
   But I also know that airflow listens to sigterm signal and stops the task on it. So I am not able to use PythonOperator with multiprocessing. It is quite annoying.
   
   The workaround is to wrap python code with multiprocessing to BashOperator. Hope this would help someone.
   
   It would be great to be able to use PythonOperator for such things. Hope this will help to investigate this issue. I think it should be discussed and maybe reopened.


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