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/06/29 09:48:20 UTC

[GitHub] [airflow] dan-origami opened a new issue #16703: Workers silently crash after memory build up

dan-origami opened a new issue #16703:
URL: https://github.com/apache/airflow/issues/16703


   **Apache Airflow version**: 2.0.2
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`): 1.18.15
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**: AWS
   - **OS** (e.g. from /etc/os-release): Ubuntu 20.04
   - **Kernel** (e.g. `uname -a`): Linux 5.4.0-1024-aws #24-Ubuntu
   - **Install tools**: Dockerfile
   - **Others**: Custom Dockerfile (not official airflow image from dockerhub)
                           Celery Workers
   
   **What happened**:
   
   Memory usage builds up on our celery worker pods until they silently crash. Resource usage flat lines and no logs are created by the worker. The process is still running and Celery (verified via ping and flower) thinks the workers are up and running. 
   No tasks are finished by Airflow, the schedulers are running fine and still logging appropriately but the workers are doing nothing. Workers do not accept any tasks and inflight jobs hang.
   They do not log an error message and the pod is not restarted as the process hasn't crashed.
   Our workers do not all crash at the same time, it happens over a couple of hours even if they were all restarted at the same time, so it seems to be related to how many jobs the worker has done/logs/other-non-time event.
   
   I believe this is related to the logs generated by the workers, Airflow appears to be reading in the existing log files to memory. Memory usage drops massively when the log files are deleted and then resume to build up again. 
   
   There doesn't appear to be a definite upper limit of memory that the pod hits when it crashes, but its around the 8 or 10GB mark (there is 14 available to the pods but they dont hit that).
   
   Log size on disk correlates to more memory usage by a worker pod than one with smaller log size on disk.
   
   **What you expected to happen**:
   
   If the worker has crashed/ceased functioning it should either be able to log an appropriate message if the process is up or crash cleanly and be able to be restarted.
   Existing log files should not contribute to the memory usage of the airflow process either.
   Celery should also be able to detect that the worker is no longer functional.
    
   **How to reproduce it**:
   
   Run an airflow cluster with 40+ DAGs with several hundred tasks in total in an environment that has observable metrics, we use k8s with Prometheus.
   We have 5x worker pods.
   Monitor the memory usage of the worker containers/pods over time as well as the size of the airflow task logs. The trend should only increase.
   
   **Anything else we need to know**:
   
   This problem occurs constantly, after a clean deployment and in multiple environments.
   The official Airflow docker image contains a [log-cleaner](https://github.com/apache/airflow/blob/main/scripts/in_container/prod/clean-logs.sh) so its possible this has been avoided but in general 15 days default would be far too long. Our workers crash between 2 or 3 days. 
   Resorting to an aggressive log cleaning script has mitigated the problem for us but without proper error logs or reasons for the crash it hard to be definite that we are safe.
   
   This is our airflow.cfg logging config, we aren't doing anything radical just storing in a bucket.
   
   
   ```
   [logging]
   # Airflow can store logs remotely in AWS S3, Google Cloud Storage or Elastic Search.
   # Users must supply an Airflow connection id that provides access to the storage
   # location. If remote_logging is set to true, see UPDATING.md for additional
   # configuration requirements.
   
   # remote_logging = $ENABLE_REMOTE_LOGGING
   # remote_log_conn_id = s3conn
   # remote_base_log_folder = $LOGS_S3_BUCKET
   # encrypt_s3_logs = False
   
   remote_logging = True
   remote_log_conn_id = s3conn
   remote_base_log_folder = $AIRFLOW__CORE__REMOTE_BASE_LOG_FOLDER
   encrypt_s3_logs = False
   
   # Log format
   log_format = [%%(asctime)s] {%%(filename)s:%%(lineno)d} %%(levelname)s - %%(message)s
   simple_log_format = %%(asctime)s %%(levelname)s - %%(message)s
   
   # Logging level
   logging_level = INFO
   
   # Logging class
   # Specify the class that will specify the logging configuration
   # This class has to be on the python classpath
   logging_config_class =
   
   # The folder where airflow should store its log files
   # This path must be absolute
   base_log_folder = /usr/local/airflow/logs
   
   # Name of handler to read task instance logs.
   # Default to use file task handler.
   # task_log_reader = file.task
   task_log_reader = task
   ```
   
   Here is a memory usage graph of a crashed worker pod, the flat line is when it is in a crashed state and then restarted. There is also a big cliff on the right of the graph at about 0900 on June 29th where I manually cleaned the log files from the disk.
   
   ![Crashed airflow worker](https://i.imgur.com/mO2ecQO.png)
   
   
   The last few log lines before it crashed:
   
   ```
   Jun 25, 2021 @ 04:28:01.831 | [2021-06-25 03:28:01,830: INFO/MainProcess] Received task: airflow.executors.celery_executor.execute_command[5f802ffb-d5af-40ae-9e99-5e0501bf7d1c]
   
    Jun 25, 2021 @ 04:27:36.769 | [2021-06-25 03:27:36,769: INFO/MainProcess] Received task: airflow.executors.celery_executor.execute_command[737d4310-c6ae-450f-889a-ffee53e94d33]
   
     Jun 25, 2021 @ 04:27:25.565 | [2021-06-25 03:27:25,564: WARNING/ForkPoolWorker-13] Running <TaskInstance: a_task_name 2021-06-25T02:18:00+00:00 [queued]> on host airflow-worker-3.airflow-worker.airflow.svc.cluster.local
   
     Jun 25, 2021 @ 04:27:25.403 | [2021-06-25 03:27:25,402: INFO/ForkPoolWorker-13] Filling up the DagBag from /usr/local/airflow/dags/a_dag.py
   
     Jun 25, 2021 @ 04:27:25.337 | [2021-06-25 03:27:25,337: INFO/ForkPoolWorker-13] Executing command in Celery: ['airflow', 'tasks', 'run', 'task_name_redacted', 'task, '2021-06-25T02:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', '/usr/local/airflow/dags/a_dag.py']
   
     Jun 25, 2021 @ 04:27:25.327 | [2021-06-25 03:27:25,326: INFO/ForkPoolWorker-13] Task airflow.executors.celery_executor.execute_command[4d9ee684-4ae3-41d2-8a00-e8071179a1b1] succeeded in 5.212706514168531s: None
   
     Jun 25, 2021 @ 04:27:24.980 | [2021-06-25 03:27:24,979: INFO/ForkPoolWorker-13] role_arn is None
   
      Jun 25, 2021 @ 04:27:24.968 | [2021-06-25 03:27:24,968: INFO/ForkPoolWorker-13] No credentials retrieved from Connection
   
      Jun 25, 2021 @ 04:27:24.968 | [2021-06-25 03:27:24,968: INFO/ForkPoolWorker-13] Creating session with aws_access_key_id=None region_name=None
   
      Jun 25, 2021 @ 04:27:24.954 | [2021-06-25 03:27:24,953: INFO/ForkPoolWorker-13] Airflow Connection: aws_conn_id=s3conn
   
     Jun 25, 2021 @ 04:27:20.610 | [2021-06-25 03:27:20,610: WARNING/ForkPoolWorker-13] Running <TaskInstance: task_name_redacted 2021-06-25T03:10:00+00:00 [queued]> on host airflow-worker-3.airflow-worker.airflow.svc.cluster.local
   ```
   
   
   


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131501559-6448a861-295c-467e-816e-3c010ed3187b.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131501626-b86d8593-ae13-41b3-8a85-1876213eb3bc.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131501644-cc2a1779-5bf9-45fd-85b2-a180ed96f98b.png">
   


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-873920393


   FWIW; I've just noticed this happened on a worker pod over the weekend that was not experiencing abnormal memory usage or build ups. 
   
   It does seem to still log occasionally with a celery sync.
   
   ```
   [2021-07-04 16:56:50,297: INFO/ForkPoolWorker-14] Executing command in Celery: ['airflow', 'tasks', 'run', 'a_task', 'api', '2021-07-04T16:22:00+00:00', '--local', '--pool', 'default_pool', '--subdir', '/usr/local/airflow/dags/dag.py']
   [2021-07-04 16:56:50,415: INFO/ForkPoolWorker-14] Filling up the DagBag from /usr/local/airflow/dags/dag.py
   [2021-07-04 16:56:50,691: WARNING/ForkPoolWorker-14] Running <TaskInstance: task task 2021-07-04T16:22:00+00:00 [queued]> on host airflow-worker-0.airflow-worker.airflow.svc.cluster.local
   [2021-07-05 03:10:55,663: INFO/MainProcess] sync with celery@airflow-worker-1
   [2021-07-05 08:13:04,899: INFO/MainProcess] sync with celery@airflow-worker-1
   ```
   
   I will probably upgrade these airflows to 2.1 to see if that makes any difference


-- 
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] boring-cyborg[bot] commented on issue #16703: Workers silently crash after memory build up

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


   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.

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

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



[GitHub] [airflow] ashb commented on issue #16703: Workers silently crash after memory build up

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






-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131503555-c0f07c20-0241-4865-b3dc-4fef919c1eea.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131503356-2aafb596-e90e-4a66-b0a4-aaf73073c91d.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131503371-ca565fe7-9bb2-43d2-89fc-2929f89c71d4.png">
   
   
   


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-873920393


   FWIW; I've just noticed this happened on a worker pod over the weekend that was not experiencing abnormal memory usage or build ups. 
   
   It does seem to still log occasionally with a celery sync.
   
   ```
   [2021-07-04 16:56:50,297: INFO/ForkPoolWorker-14] Executing command in Celery: ['airflow', 'tasks', 'run', 'a_task', 'api', '2021-07-04T16:22:00+00:00', '--local', '--pool', 'default_pool', '--subdir', '/usr/local/airflow/dags/dag.py']
   [2021-07-04 16:56:50,415: INFO/ForkPoolWorker-14] Filling up the DagBag from /usr/local/airflow/dags/dag.py
   [2021-07-04 16:56:50,691: WARNING/ForkPoolWorker-14] Running <TaskInstance: task task 2021-07-04T16:22:00+00:00 [queued]> on host airflow-worker-0.airflow-worker.airflow.svc.cluster.local
   [2021-07-05 03:10:55,663: INFO/MainProcess] sync with celery@airflow-worker-1
   [2021-07-05 08:13:04,899: INFO/MainProcess] sync with celery@airflow-worker-1
   ```


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767






-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131501559-6448a861-295c-467e-816e-3c010ed3187b.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131501626-b86d8593-ae13-41b3-8a85-1876213eb3bc.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131501644-cc2a1779-5bf9-45fd-85b2-a180ed96f98b.png">
   


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-887447309


   If you want us to try 2.1.2 as well we can definitely do that.


-- 
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] ashb commented on issue #16703: Workers silently crash after memory build up

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


   Could you also check the output of `dmesg` on the host (likely won't have info from two days ago now, so might need to trigger this again) at around the time the worker "stalls" -- specifically I am interested to see if there is anything in the logs related to the linux OOM killer being triggered.


-- 
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] kaxil commented on issue #16703: Workers silently crash after memory build up

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


   @dan-origami Were you able to upgrade to latest 2.1 release and see if this is still the issue


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870525292


   using `container_memory_working_set_bytes`
   
   ```
   sum(container_memory_working_set_bytes{pod=~"airflow-worker-.*",pod!="",container!="POD",image!="",container!="statsd"}) by (pod)
   ```
   
   Working bytes seems to grow but roughly similar across all workers.
   (Bad graph title)
   
   Worker-3
   ![worker-3 as above](https://i.imgur.com/cdbedwB.png)
   
   All workers
   ![All workers](https://i.imgur.com/KrMKJy6.png)
   
   
   And now using 
   `container_memory_rss`
   
   ```
   ```
   
   ![worker-3](https://i.imgur.com/uax7DI0.png)
   
   ![all workers](https://i.imgur.com/AFhDmdT.png)
   
   
   I've also checked the pid files in /proc/pid/status and this seems to be reflected but I don't have the historical pid ones of course for now.


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-873920393






-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870485384


   @ashb its `container_memory_usage_bytes` with full query below
   
   ```sum(container_memory_usage_bytes{container!="POD",pod=~"airflow.*",image!=""}) by (pod)```
   
   Nothing on the dmesg at all, i checked and due to our affinity/selector the pod was always on the same server 
   
   ```
   [Tue Jun 22 13:05:51 2021] device veth7323bc8f entered promiscuous mode
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered blocking state
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered forwarding state
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:10 2021] device veth742b6e37 left promiscuous mode
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:19 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethab64c6d9: link becomes ready
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered disabled state
   [Tue Jun 22 13:08:19 2021] device vethab64c6d9 entered promiscuous mode
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered forwarding state
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Tue Jun 22 14:57:36 2021] device vethda1d06e5 left promiscuous mode
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Fri Jun 25 09:38:38 2021] device vethfa6a98a7 left promiscuous mode
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Mon Jun 28 08:43:32 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethae505a5a: link becomes ready
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered disabled state
   [Mon Jun 28 08:43:32 2021] device vethae505a5a entered promiscuous mode
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered forwarding state
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   [Mon Jun 28 08:43:40 2021] device veth7323bc8f left promiscuous mode
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   ```
   
   ![Verification of node staying the same](https://i.imgur.com/T5MFNwy.png)


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870485384


   @ashb its `container_memory_usage_bytes` with full query below
   
   ```sum(container_memory_usage_bytes{container!="POD",pod=~"airflow.*",image!=""}) by (pod)``` I had just the worker selected for that view of course
   
   Nothing on the dmesg at all, i checked and due to our affinity/selector the pod was always on the same server 
   
   ```
   [Tue Jun 22 13:05:51 2021] device veth7323bc8f entered promiscuous mode
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered blocking state
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered forwarding state
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:10 2021] device veth742b6e37 left promiscuous mode
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:19 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethab64c6d9: link becomes ready
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered disabled state
   [Tue Jun 22 13:08:19 2021] device vethab64c6d9 entered promiscuous mode
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered forwarding state
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Tue Jun 22 14:57:36 2021] device vethda1d06e5 left promiscuous mode
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Fri Jun 25 09:38:38 2021] device vethfa6a98a7 left promiscuous mode
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Mon Jun 28 08:43:32 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethae505a5a: link becomes ready
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered disabled state
   [Mon Jun 28 08:43:32 2021] device vethae505a5a entered promiscuous mode
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered forwarding state
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   [Mon Jun 28 08:43:40 2021] device veth7323bc8f left promiscuous mode
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   ```
   
   ![Verification of node staying the same](https://i.imgur.com/T5MFNwy.png)


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131503343-f463ccea-f8f4-45cb-a50d-d9f02b3ba2dc.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131503356-2aafb596-e90e-4a66-b0a4-aaf73073c91d.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131503371-ca565fe7-9bb2-43d2-89fc-2929f89c71d4.png">
   
   
   


-- 
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] boring-cyborg[bot] commented on issue #16703: Workers silently crash after memory build up

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


   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.

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

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



[GitHub] [airflow] potiuk commented on issue #16703: Workers silently crash after memory build up

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


   Nice analysis. Will help us to investigate. Thanks @dan-origami !


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-873920393


   FWIW; I've just noticed this happened on a worker pod over the weekend that was not experiencing abnormal memory usage or build ups. 
   
   It does seem to still log occasionally with a celery sync.
   
   ```
   [2021-07-04 16:56:50,297: INFO/ForkPoolWorker-14] Executing command in Celery: ['airflow', 'tasks', 'run', 'a_task', 'api', '2021-07-04T16:22:00+00:00', '--local', '--pool', 'default_pool', '--subdir', '/usr/local/airflow/dags/dag.py']
   [2021-07-04 16:56:50,415: INFO/ForkPoolWorker-14] Filling up the DagBag from /usr/local/airflow/dags/dag.py
   [2021-07-04 16:56:50,691: WARNING/ForkPoolWorker-14] Running <TaskInstance: task bmrs_dersysdata_spark.api 2021-07-04T16:22:00+00:00 [queued]> on host airflow-worker-0.airflow-worker.airflow.svc.cluster.local
   [2021-07-05 03:10:55,663: INFO/MainProcess] sync with celery@airflow-worker-1
   [2021-07-05 08:13:04,899: INFO/MainProcess] sync with celery@airflow-worker-1
   ```


-- 
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] ashb commented on issue #16703: Workers silently crash after memory build up

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


   Oh one extra thing -- are you able to check the RSS of the processes in the container and see if they are all growing equally, if one is clearly using more than another, or if _no_ process's RSS actually shows any growth?
   
   (I've been trying to track down a different memory issue in the scheduler where the working_set_bytes is growing, but no process shows the RSS, and would like to check they are different behaviours.)


-- 
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] potiuk commented on issue #16703: Workers silently crash after memory build up

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


   Nice analysis. Will help us to investigate. Thanks @dan-origami !


-- 
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] kaxil commented on issue #16703: Workers silently crash after memory build up

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


   Yes please


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870485384






-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-874024816


   Around the time it failed i see some tmp files /tmp
   
   ```
   -rw-------  1 airflow   89 Jul  4 15:47 tmpai_jn6o_
   -rw-------  1 airflow   89 Jul  4 16:47 tmpyqxc49z8
   -rw-------  1 airflow   89 Jul  4 17:17 tmpuyxaiqwj
   ```
   
   ```
   airflow@airflow-worker-0:/tmp$ cat tmpuyxaiqwj
   ��N�airflow.exceptions��AirflowException����Task received SIGTERM signal���R�.airflow@airflow-worker-0:/tmp$
   ```
   
   There is a bunch of these appearing over time on the workers, again hard to know if its related.


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767






-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-895051127


   @ephraimbuddy I am trying it this week


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-887446870


   @kaxil we are on 2.1.1 at the moment and it seems better, there were some fixes listed in the release for CeleryExecutors so went for it. I can't definitely say that its fixed though as we do churn our airflows quite a lot at the moment so it doesn't always get prolonged runtime without being redeployed. 
   
   I see this issue is added for 2.1.3 do you know if there is anything specific that has been found around 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.

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

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



[GitHub] [airflow] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870485384


   @ashb its `container_memory_usage_bytes` with full query below
   
   ```sum(container_memory_usage_bytes{container!="POD",pod=~"airflow.*",image!=""}) by (pod)``` I had just the worker selected for that view of course
   
   Nothing on the dmesg at all, i checked and due to our affinity/selector the pod was always on the same server 
   
   ```
   [Tue Jun 22 13:05:51 2021] device veth7323bc8f entered promiscuous mode
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered blocking state
   [Tue Jun 22 13:05:51 2021] cni0: port 6(veth7323bc8f) entered forwarding state
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:10 2021] device veth742b6e37 left promiscuous mode
   [Tue Jun 22 13:08:10 2021] cni0: port 2(veth742b6e37) entered disabled state
   [Tue Jun 22 13:08:19 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethab64c6d9: link becomes ready
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered disabled state
   [Tue Jun 22 13:08:19 2021] device vethab64c6d9 entered promiscuous mode
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered blocking state
   [Tue Jun 22 13:08:19 2021] cni0: port 2(vethab64c6d9) entered forwarding state
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Tue Jun 22 14:57:36 2021] device vethda1d06e5 left promiscuous mode
   [Tue Jun 22 14:57:36 2021] cni0: port 5(vethda1d06e5) entered disabled state
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Fri Jun 25 09:38:38 2021] device vethfa6a98a7 left promiscuous mode
   [Fri Jun 25 09:38:38 2021] cni0: port 3(vethfa6a98a7) entered disabled state
   [Mon Jun 28 08:43:32 2021] IPv6: ADDRCONF(NETDEV_CHANGE): vethae505a5a: link becomes ready
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered disabled state
   [Mon Jun 28 08:43:32 2021] device vethae505a5a entered promiscuous mode
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered blocking state
   [Mon Jun 28 08:43:32 2021] cni0: port 3(vethae505a5a) entered forwarding state
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   [Mon Jun 28 08:43:40 2021] device veth7323bc8f left promiscuous mode
   [Mon Jun 28 08:43:40 2021] cni0: port 6(veth7323bc8f) entered disabled state
   ```
   
   ![Verification of node staying the same](https://i.imgur.com/T5MFNwy.png)


-- 
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] ephraimbuddy commented on issue #16703: Workers silently crash after memory build up

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


   Hi @dan-origami, have you tried airflow 2.1.2?


-- 
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] ashb commented on issue #16703: Workers silently crash after memory build up

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


   @dan-origami Hmmm not good!
   
   What specific metric is it that you are showing in your graph please?


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-870493524


   No problem, happy to provide full Dockerfile, metrics or anything else you want 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.

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

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



[GitHub] [airflow] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131501559-6448a861-295c-467e-816e-3c010ed3187b.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131501626-b86d8593-ae13-41b3-8a85-1876213eb3bc.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131501644-cc2a1779-5bf9-45fd-85b2-a180ed96f98b.png">
   


-- 
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] dan-origami edited a comment on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami edited a comment on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-909191767


    @kaxil @ephraimbuddy Just to give you a bit of an update, I think I have found what the actual cause of this is.
   
   I noticed that we seem to hit a problem with the number of Active Tasks on a Celery Worker (all our settings here are currently default) so max 16 per worker and 32 across the airflow setup.
   
   However I noticed that when this problem manifests we don't schedule anything so started looking into our workers via Flower.
   
   Screenshots are below, but basically we have these fairly big DAGs that run some spark jobs on a spark cluster in the same kubernetes cluster (pyspark, so the driver exists as part of the airflow worker, we can do more details on spark if you want but its BashOperator and not SparkOperator for a number of reasons).
   
   Sometimes the tasks in these DAGs fail, which is picked up by Airflow as the task is marked as Failed. However these tasks sit on the Celery worker as an active task still and are not removed.
   
   We can manually delete them and it works, so the celery worker itself is still active and not crashed. They just do not seem to log anything when they are not picking up any new tasks/running them. Active PIDs etc as listed in Flower also seem to match up.
   
   It's not clear why the Task failed but  we have the logs of it being picked up by the Worker (i've removed a few bits).
   
   It also explains why I was down the memory/resource issue rabbithole as these tasks sit around on the worker(s).
   
   There are some parameters that we can tune I think to include timeouts on the tasks and stuff on the Celery side, do you know if there is any known issues with this disconnect between a Failed Task in Airflow and it not being removed from the Celery Worker?
   
   The worker was not rebooted/crashed at any point during this time.
   
   Also this investigation was carried out today (31st Aug) and all the dates for stuff stuck since the 28th is correct that its been there for over 4 days.
   
   ```
   [2021-08-28 06:54:43,865: INFO/ForkPoolWorker-8] Executing command in Celery: ['airflow', 'tasks', 'run', 'dagname', 'timeseries_api', '2021-08-28T06:18:00+00:00', '--local', '--pool', 'default_pool', '--subdir', 'dagfolder']
   [2021-08-28 06:54:44,122: WARNING/ForkPoolWorker-8] Running <TaskInstance:dagname.timeseries_api 2021-08-28T06:18:00+00:00 [queued]> on host airflow-worker-2.airflow-worker.airflow.svc.cluster.local
   ```
   
   <img width="579" alt="Screenshot 2021-08-31 at 13 16 40" src="https://user-images.githubusercontent.com/51330721/131501535-c74321ef-04a1-46ba-b260-032ac52089f0.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 17 18" src="https://user-images.githubusercontent.com/51330721/131503555-c0f07c20-0241-4865-b3dc-4fef919c1eea.png">
   
   <img width="1101" alt="Screenshot 2021-08-31 at 13 18 21" src="https://user-images.githubusercontent.com/51330721/131503356-2aafb596-e90e-4a66-b0a4-aaf73073c91d.png">
   
   <img width="1666" alt="Screenshot 2021-08-31 at 13 18 03" src="https://user-images.githubusercontent.com/51330721/131503371-ca565fe7-9bb2-43d2-89fc-2929f89c71d4.png">
   
   
   


-- 
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] dan-origami commented on issue #16703: Workers silently crash after memory build up

Posted by GitBox <gi...@apache.org>.
dan-origami commented on issue #16703:
URL: https://github.com/apache/airflow/issues/16703#issuecomment-873920393






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