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/01/18 17:37:37 UTC

[GitHub] [airflow] bkumarac20 opened a new issue #13747: Airflow task is marked as zombie , when the backend process on remote server is still running.

bkumarac20 opened a new issue #13747:
URL: https://github.com/apache/airflow/issues/13747


   **Apache Airflow version**:1.10.9-composer
   
   **Kubernetes Version** : Client Version: version.Info{Major:"1", Minor:"15+", GitVersion:"v1.15.12-gke.6002", GitCommit:"035184604aff4de66f7db7fddadb8e7be76b6717", GitTreeState:"clean", BuildDate:"2020-12-01T23:13:35Z", GoVersion:"go1.12.17b4", Compiler:"gc", Platform:"linux/amd64"}
   
   **Environment:** Airflow, running on top of Kubernetes - Linux version 4.19.112
   
   - OS : Linux version 4.19.112+ (builder@7fc5cdead624) (Chromium OS 9.0_pre361749_p20190714-r4 clang version 9.0.0 (/var/cache/chromeos-cache/distfiles/host/egit-src/llvm-project c11de5eada2decd0a495ea02676b6f4838cd54fb) (based on LLVM 9.0.0svn)) #1 SMP Fri Sep 4 12:00:04 PDT 2020
   - Kernel : Linux gke-europe-west2-asset-c-default-pool-dc35e2f2-0vgz 4.19.112+ #1 SMP Fri Sep 4 12:00:04 PDT 2020 x86_64 Intel(R) Xeon(R) CPU @ 2.20GHz GenuineIntel GNU/Linux
   
   **What happened ?**
   A running task is marked as Zombie after the execution time crossed the latest heartbeat + 5 minutes.
   The task is running in background in another application server, triggered using SSHOperator.
   `[2021-01-18 11:53:37,491] {taskinstance.py:888} INFO - Executing <Task(SSHOperator): load_trds_option_composite_file> on 2021-01-17T11:40:00+00:00
   [2021-01-18 11:53:37,495] {base_task_runner.py:131} INFO - Running on host: airflow-worker-6f6fd78665-lm98m
   [2021-01-18 11:53:37,495] {base_task_runner.py:132} INFO - Running: ['airflow', 'run', 'dsp_etrade_process_trds_option_composite_0530', 'load_trds_option_composite_file', '2021-01-17T11:40:00+00:00', '--job_id', '282759', '--pool', 'default_pool', '--raw', '-sd', 'DAGS_FOLDER/dsp_etrade_trds_option_composite_0530.py', '--cfg_path', '/tmp/tmpge4_nva0']`
   
   **Task Executing time:**
   `dag_id		dsp_etrade_process_trds_option_composite_0530
   duration	7270.47
   start_date	2021-01-18 11:53:37,491
   end_date	2021-01-18 13:54:47.799728+00:00`
   
   **Scheduler Logs during that time:**
   `[2021-01-18 13:54:54,432] {taskinstance.py:1135} ERROR - <TaskInstance: dsp_etrade_process_etrd.push_run_date 2021-01-18 13:30:00+00:00 [running]> detected as zombie
   {
   textPayload: "[2021-01-18 13:54:54,432] {taskinstance.py:1135} ERROR - <TaskInstance: dsp_etrade_process_etrd.push_run_date 2021-01-18 13:30:00+00:00 [running]> detected as zombie"
   insertId: "1ca8zyfg3zvma66"
   resource: {
   type: "cloud_composer_environment"
   labels: {3}
   }
   timestamp: "2021-01-18T13:54:54.432862699Z"
   severity: "ERROR"
   logName: "projects/asset-control-composer-prod/logs/airflow-scheduler"
   receiveTimestamp: "2021-01-18T13:54:55.714437665Z"
   }
   `
   
   **Airflow-webserver log :**
   `X.X.X.X - - [18/Jan/2021:13:54:39 +0000] "GET /_ah/health HTTP/1.1" 200 187 "-" "GoogleHC/1.0"
   {
   textPayload: "172.17.0.5 - - [18/Jan/2021:13:54:39 +0000] "GET /_ah/health HTTP/1.1" 200 187 "-" "GoogleHC/1.0"
   "
   insertId: "1sne0gqg43o95n3"
   resource: {2}
   timestamp: "2021-01-18T13:54:45.401670481Z"
   logName: "projects/asset-control-composer-prod/logs/airflow-webserver"
   receiveTimestamp: "2021-01-18T13:54:50.598807514Z"
   }
   `
   
   **Airflow Info logs :**
   `2021-01-18 08:54:47.799 EST
   {
   textPayload: "NoneType: None
   "
   insertId: "1ne3hqgg47yzrpf"
   resource: {2}
   timestamp: "2021-01-18T13:54:47.799661030Z"
   severity: "INFO"
   logName: "projects/asset-control-composer-prod/logs/airflow-scheduler"
   receiveTimestamp: "2021-01-18T13:54:50.914461159Z"
   }
   
   [2021-01-18 13:54:47,800] {taskinstance.py:1192} INFO - Marking task as FAILED.dag_id=dsp_etrade_process_trds_option_composite_0530, task_id=load_trds_option_composite_file, execution_date=20210117T114000, start_date=20210118T115337, end_date=20210118T135447
   Copy link
   {
   textPayload: "[2021-01-18 13:54:47,800] {taskinstance.py:1192} INFO - Marking task as FAILED.dag_id=dsp_etrade_process_trds_option_composite_0530, task_id=load_trds_option_composite_file, execution_date=20210117T114000, start_date=20210118T115337, end_date=20210118T135447"
   insertId: "1ne3hqgg47yzrpg"
   resource: {2}
   timestamp: "2021-01-18T13:54:47.800605248Z"
   severity: "INFO"
   logName: "projects/asset-control-composer-prod/logs/airflow-scheduler"
   receiveTimestamp: "2021-01-18T13:54:50.914461159Z"
   }
   `
   
   **Airflow Database shows the latest heartbeat as:**
   select state, latest_heartbeat from job where id=282759
   --------------------------------------
   state	| latest_heartbeat
   running	| 2021-01-18 13:48:41.891934
   
   **Airflow Configurations:**
   celery
   worker_concurrency=6
   
   scheduler
   scheduler_health_check_threshold=60
   scheduler_zombie_task_threshold=300
   max_threads=2
   
   core
   dag_concurrency=6
   
   **Kubernetes Cluster :**
   Worker nodes : 6
   
   What we expected to happen ?
   The backend process takes around 2hrs 30 minutes to finish. During such long running jobs the task is detected as zombie.
   Even though the worker node is still processing the task. The state of the job is still marked as 'running'. State of the task is not known during the run time.
   
   **How to reproduce it:**
   
   - Issue occurs sporadically, thus challenging to deterministically reproduce.
   - We encountered it using atleast: SSHOperator
   
   


----------------------------------------------------------------
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 #13747: Airflow task is marked as zombie , when the backend process on remote server is still running.

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


   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] eladkal commented on issue #13747: Airflow task is marked as zombie , when the backend process on remote server is still running.

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


   This issue is reported against old version of airflow which is end of life.
   If the issue still happens in latest airflow version please let us know & provide updated 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.

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

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



[GitHub] [airflow] eladkal closed issue #13747: Airflow task is marked as zombie , when the backend process on remote server is still running.

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


   


-- 
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] bkumarac20 commented on issue #13747: Airflow task is marked as zombie , when the backend process on remote server is still running.

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


   Hello Team,
   
   Debugging further, I have noticed a pattern where the worker is not sending heartbeat to db. When the time difference between the scheduler current time & latest heart is more than 5 minutes + X seconds, the task is detected as zombie.
   
   
   [dsp_etrade_process_trds_forex_0530.txt](https://github.com/apache/airflow/files/5842723/dsp_etrade_process_trds_forex_0530.txt)
   [dsp_etrade_process_trds_future.txt](https://github.com/apache/airflow/files/5842724/dsp_etrade_process_trds_future.txt)
   [dsp_etrade_process_trds_mutual_fund.txt](https://github.com/apache/airflow/files/5842725/dsp_etrade_process_trds_mutual_fund.txt)
   
   


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