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/05/12 08:09:07 UTC

[GitHub] [airflow] c-thiel opened a new issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

c-thiel opened a new issue #15793:
URL: https://github.com/apache/airflow/issues/15793


   **Apache Airflow version**: 2.0.2
   
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`): 1.17.14
   
   **Environment**:
   
   - **Helm-Chart**: https://github.com/airflow-helm/charts/tree/main/charts/airflow
   - **Executor**: CeleryExector on Kubernetes
   - **Operator**: Mostly KubernetesPodOperator, most tasks take 5 or 10 pool slots
   - **Parallelism**: Very high (1000)
   - **AIRFLOW__SCHEDULER__PARSING_PROCESSES**: 1
   - **AIRFLOW__CELERY__WORKER_PREFETCH_MULTIPLIER**: 1
   - **scheduler pods**: 1
   
   **What happened**:
   
   The scheduler queues more slots than are available in the pod:
   ![image](https://user-images.githubusercontent.com/12560027/117936399-185eb380-b305-11eb-9366-9c9cf65ac8ac.png)
   
   As a result (I think), 80% of the Tasks fail:
   ```
   *** Reading remote log from s3://wyrm-logs/airflow/endurance-wyrm-dev/logs/DAG-ID/my_task/2021-05-11T07:12:49.813565+00:00/2.log.
   [2021-05-12 07:31:29,247] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: DAG-ID.my_task 2021-05-11T07:12:49.813565+00:00 [queued]>
   [2021-05-12 07:31:29,261] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: DAG-ID.my_task 2021-05-11T07:12:49.813565+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -70, 'kubernetes', 5)
   [2021-05-12 07:31:29,261] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 07:31:29,261] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 2 of 3. State set to NONE.
   [2021-05-12 07:31:29,261] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 07:31:29,287] {local_task_job.py:93} INFO - Task is not able to be run
   
   [2021-05-12 07:31:34,003] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: DAG-ID.my_task 2021-05-11T07:12:49.813565+00:00 [queued]>
   [2021-05-12 07:31:34,017] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: DAG-ID.my_task 2021-05-11T07:12:49.813565+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -115, 'kubernetes', 5)
   [2021-05-12 07:31:34,017] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 07:31:34,017] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 2 of 3. State set to NONE.
   [2021-05-12 07:31:34,017] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 07:31:34,027] {local_task_job.py:93} INFO - Task is not able to be run
   ```
   The Task is in the failed state afterwards.
   
   The number of ``Rescheduling due to concurrency limits reached`` messages depends from task to task.
   
   
   **What you expected to happen**:
   
   The task is Rescheduled until pool slots are available. Then it runs. It does not fail due to depleted pool slots.
   
   **How to reproduce it**:
   
   Increase Parrallelism to a very high number, create a pool with only very few slots.
   
   --- I will try to add an example based on MiniKube later today ---
   
   **What I think what happened**
   
   Firstly the scheduler is for some reason queuing more tasks than there are slots in the pool - I think it is simply ignoring the queued slots. The executors, beeing capable of depleting the pool instantly, are then just pulling too many tasks of the too long queue. The last tasks notice that the pool is depleted and mark them as failed.
   
   I am not too deep in the code - but I see two problems here: The scheduler is queuing more tasks than slots are available and the worker (probably) mark the tasks as failed if the pool is depleted instead of rescheduling it.
   


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

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



[GitHub] [airflow] c-thiel commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
c-thiel commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-839662753


   @ashb I added the instructions above.
   Please find following some Screenshots from the setup:
   ![image](https://user-images.githubusercontent.com/12560027/117960728-962eb900-b31d-11eb-9793-180881fff1b9.png)
   
   This is the log of one of the python tasks:
   ```
   *** Reading local file: /opt/airflow/logs/test-python/python-185/2021-05-11T00:00:00+00:00/1.log
   [2021-05-12 10:16:05,723] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:16:06,020] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:16:06,021] {taskinstance.py:1068} INFO - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:16:06,021] {taskinstance.py:1069} INFO - Starting attempt 1 of 1
   [2021-05-12 10:16:06,021] {taskinstance.py:1070} INFO - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:16:06,629] {taskinstance.py:1089} INFO - Executing <Task(PythonOperator): python-185> on 2021-05-11T00:00:00+00:00
   [2021-05-12 10:16:06,720] {standard_task_runner.py:52} INFO - Started process 126 to run task
   [2021-05-12 10:16:06,727] {standard_task_runner.py:76} INFO - Running: ['airflow', 'tasks', 'run', 'test-python', 'python-185', '2021-05-11T00:00:00+00:00', '--job-id', '218', '--pool', 'default_pool', '--raw', '--subdir', 'DAGS_FOLDER/dag.py', '--cfg-path', '/tmp/tmpagjtm408', '--error-file', '/tmp/tmp8l_qm7bp']
   [2021-05-12 10:16:06,727] {standard_task_runner.py:77} INFO - Job 218: Subtask python-185
   [2021-05-12 10:16:13,428] {logging_mixin.py:104} INFO - Running <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [running]> on host airflow-worker-1.airflow-worker.default.svc.cluster.local
   [2021-05-12 10:16:15,824] {taskinstance.py:1281} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=test-python
   AIRFLOW_CTX_TASK_ID=python-185
   AIRFLOW_CTX_EXECUTION_DATE=2021-05-11T00:00:00+00:00
   AIRFLOW_CTX_DAG_RUN_ID=scheduled__2021-05-11T00:00:00+00:00
   [2021-05-12 10:16:35,921] {python.py:118} INFO - Done. Returned value was: None
   [2021-05-12 10:16:36,032] {taskinstance.py:1185} INFO - Marking task as SUCCESS. dag_id=test-python, task_id=python-185, execution_date=20210511T000000, start_date=20210512T101605, end_date=20210512T101636
   [2021-05-12 10:16:36,434] {taskinstance.py:1246} INFO - 0 downstream tasks scheduled from follow-on schedule check
   [2021-05-12 10:16:36,621] {local_task_job.py:146} INFO - Task exited with return code 0
   [2021-05-12 10:19:45,235] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:19:45,330] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -10, 'kubernetes', 5)
   [2021-05-12 10:19:45,421] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:45,421] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 1 of 1. State set to NONE.
   [2021-05-12 10:19:45,421] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:45,430] {local_task_job.py:93} INFO - Task is not able to be run
   [2021-05-12 10:19:47,725] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:19:47,735] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -10, 'kubernetes', 5)
   [2021-05-12 10:19:47,823] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:47,823] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 1 of 1. State set to NONE.
   [2021-05-12 10:19:47,823] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:47,835] {local_task_job.py:93} INFO - Task is not able to be run
   [2021-05-12 10:19:49,530] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:19:49,626] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -10, 'kubernetes', 5)
   [2021-05-12 10:19:49,727] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:49,727] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 1 of 1. State set to NONE.
   [2021-05-12 10:19:49,727] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:49,736] {local_task_job.py:93} INFO - Task is not able to be run
   [2021-05-12 10:19:51,839] {taskinstance.py:877} INFO - Dependencies all met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>
   [2021-05-12 10:19:52,024] {taskinstance.py:867} INFO - Dependencies not met for <TaskInstance: test-python.python-185 2021-05-11T00:00:00+00:00 [queued]>, dependency 'Pool Slots Available' FAILED: ('Not scheduling since there are %s open slots in pool %s and require %s pool slots', -10, 'kubernetes', 5)
   [2021-05-12 10:19:52,140] {taskinstance.py:1053} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:52,140] {taskinstance.py:1054} WARNING - Rescheduling due to concurrency limits reached at task runtime. Attempt 1 of 1. State set to NONE.
   [2021-05-12 10:19:52,140] {taskinstance.py:1061} WARNING - 
   --------------------------------------------------------------------------------
   [2021-05-12 10:19:52,227] {local_task_job.py:93} INFO - Task is not able to be run
   ```
   
   It is completly random which ones fail and which ones don't.


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

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



[GitHub] [airflow] ashb commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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


   Ah, there was also another fix (maybe not until 2.1.1?) where if a dynamic dag disappears when the tasks were already scheduled then they would hang around.
   
   Either way, I'm going to put the "pending-response" label on this issue so it will be auto-closed after a while if no other response.


-- 
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] c-thiel commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
c-thiel commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-839647501


   > Hmmm I wonder how this pool got over full in the first place -- did you recently upgrade from an older version?
   
   No I did not. This is a fresh install. I also just manged to reproduce on Minikube. I will add the instructions in the next hour.


-- 
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 #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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


   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] nitinpandey-154 commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-846820072


   Have been facing the same issue with 2.1.0-python3.8 docker


-- 
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] a-pertsev removed a comment on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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


   I belive the problem will be fixed by this commit:
   https://github.com/apache/airflow/pull/15426
   I faced same problems and it helped me
   


-- 
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] nitinpandey-154 commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
nitinpandey-154 commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-860373164


   @ashb - So, there was a file that was generating the dags dynamically. I started facing issues where tasks would just get queued and not execute, sometimes deleting the dag from UI would fix the issue.
   
   However, I haven't faced any issues from the past week. This was very random. My hunch is - If there is an issue in the dag generation file, the dynamic dags are not generated properly and the tasks get stuck. Since, I've handled the errors correctly and checked the dags are generated properly, haven't faced any issues lately. I Will update if anything changes.


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

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



[GitHub] [airflow] ashb commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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


   @c-thiel Thanks, we'll take a look at this


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

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



[GitHub] [airflow] a-pertsev commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
a-pertsev commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-859725991


   i belive this problem is solved in this PR:
   https://github.com/apache/airflow/pull/15426
   as @c-thiel metioned
   
   >Operator: Mostly KubernetesPodOperator, most tasks take 5 or 10 pool slots
   
   so incorrect pool slots stats could cause this kind of problems
   


-- 
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] github-actions[bot] commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-880288906


   This issue has been automatically marked as stale because it has been open for 30 days with no response from the author. It will be closed in next 7 days if no further activity occurs from the issue author.


-- 
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] a-pertsev commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
a-pertsev commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-857719984


   I belive the problem will be fixed by this commit:
   https://github.com/apache/airflow/pull/15426
   I faced same problems and it helped me
   


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

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



[GitHub] [airflow] ashb commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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


   Hmmm I wonder how this pool got over full in the first place -- did you recently upgrade from an older version?


-- 
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] github-actions[bot] commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-884574072


   This issue has been closed because it has not received response from the issue author.


-- 
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 #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

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






-- 
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] c-thiel edited a comment on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
c-thiel edited a comment on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-845204421


   @ashb today I tested with different postgres versions with no luck - the bug still persists.
   I am happy to prepare a PR for a small fix - I just couldn't identify the bug yet. I would be happy for any indication of what this could be - it's really posing us problems in production.
   
   Should we add the 2.0 Version label?
   


-- 
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] github-actions[bot] closed issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed issue #15793:
URL: https://github.com/apache/airflow/issues/15793


   


-- 
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] c-thiel commented on issue #15793: Airflow Scheduler does not take Queued Slots in Pools into consideration

Posted by GitBox <gi...@apache.org>.
c-thiel commented on issue #15793:
URL: https://github.com/apache/airflow/issues/15793#issuecomment-845204421


   @ashb today I tested with different postgres versions with no luck - the bug still persists.
   I am happy to prepare a PR for a small fix - I just couldn't identify the bug yet. I would be happy for any indication of what this could be - it's really posing us problems in production.
   
   Should we add the 2.0 Version Tag?
   


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