deepchand opened a new issue, #25981: URL: https://github.com/apache/airflow/issues/25981
### Apache Airflow version Other Airflow 2 version ### What happened Significant increase in scheduling loop time when we are crossing following parameters ``` parallelism > 60 max_active_tasks_per_dag > 60 max_active_runs_per_dag > 60 ``` We are running around 60 DAG Run of a single dag which have 10 tasks, once we cross around 60 DAG Run scheduling loop time increased upto 300 sec which is causing below warning `The scheduler does not appear to be running. Last heartbeat was received X minutes ago` and also not schedules new tasks which degrades the performance of airflow. We have are running two scheduler and both have enough resources available. We create all the dag run via rest api only. This is happening due to below function which is updating the state of the each task separately [https://github.com/apache/airflow/blob/9ac742885ffb83c15f7e3dc910b0cf9df073407a/airflow/executors/celery_executor.py#L312](url) Some of the other config settings are below which we are using ``` job_heartbeat_sec = 5 scheduler_heartbeat_sec = 5 num_runs = -1 scheduler_idle_sleep_time = 1 scheduler_health_check_threshold = 30 max_tis_per_query = 128 use_row_level_locking = True max_dagruns_to_create_per_loop = 5 max_dagruns_per_loop_to_schedule = 20 ``` <img width="1323" alt="Screenshot 2022-08-26 at 5 03 06 PM" src="https://user-images.githubusercontent.com/18530606/186894944-3337e824-2345-4078-abca-0a293eda52e3.png"> <img width="834" alt="Screenshot 2022-08-26 at 5 45 28 PM" src="https://user-images.githubusercontent.com/18530606/186901378-d2cc21ba-d320-4d3d-a8c7-254e8b0ccb8f.png"> ### What you think should happen instead As per my debugging i have seen the below function is taking significant time while updating the state of task [https://github.com/apache/airflow/blob/9ac742885ffb83c15f7e3dc910b0cf9df073407a/airflow/executors/celery_executor.py#L312](url) Although we are updating the state of tasks from below function in bulk [https://github.com/apache/airflow/blob/9ac742885ffb83c15f7e3dc910b0cf9df073407a/airflow/executors/celery_executor.py#L330](url) So my question is can we avoid updating the state of tasks from [this](https://github.com/apache/airflow/blob/9ac742885ffb83c15f7e3dc910b0cf9df073407a/airflow/executors/celery_executor.py#L312) function as it is consuming almost 95% time of a scheduling loop because anyway we are updating the state of tasks from [this]([https://github.com/apache/airflow/blob/9ac742885ffb83c15f7e3dc910b0cf9df073407a/airflow/executors/celery_executor.py#L330]) function in bulk When i did the same i have found significant decrease in scheduling loop <img width="1316" alt="Screenshot 2022-08-26 at 5 22 31 PM" src="https://user-images.githubusercontent.com/18530606/186897601-b2b23c9c-baec-40fe-bdf6-f3c126009097.png"> ### How to reproduce We can reproduce this by increasing the below parameter to parallelism > 60 max_active_tasks_per_dag > 60 max_active_runs_per_dag > 60 Dag should have atleast more then 10 tasks Note : In our case our dag is doing ssh on some remote machines and performs some package upgrade tasks on the machines ### Operating System Linux (centos 8.2.2004) ### Versions of Apache Airflow Providers ``` apache-airflow-providers-celery==2.1.0 apache-airflow-providers-ftp==2.0.1 apache-airflow-providers-http==2.0.1 apache-airflow-providers-imap==2.0.1 apache-airflow-providers-microsoft-azure==3.3.0 apache-airflow-providers-redis==2.0.1 apache-airflow-providers-slack==4.1.0 apache-airflow-providers-sqlite==2.0.1 ``` ### Deployment Other ### Deployment details ``` Airflow v2.2.2 Postgres 11 CeleryExecutor Python 3.6 ``` Constraints used - [https://raw.githubusercontent.com/apache/airflow/constraints-2.2.2/constraints-3.6.txt](url) Cluster Details - Schedulers VM (Having 16vcpu, 64GB memory)- 2 Wroker VM(16vcpu, 64GB memory) - 4 Webserver VM - 1 Some of the core config parameters ``` sql_alchemy_pool_size = 200 executor = CeleryExecutor ``` ### Anything else _No response_ ### Are you willing to submit PR? - [X] Yes I am willing to submit a PR! ### Code of Conduct - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
