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 2020/06/15 16:50:01 UTC

[GitHub] [airflow] pulsar314 opened a new issue #9316: Tasks get failed if capacity of a pool is exceeded

pulsar314 opened a new issue #9316:
URL: https://github.com/apache/airflow/issues/9316


   **Apache Airflow version**: 1.10.10 (from PyPI)
   
   
   **Environment**:
   
   - **OS**: Ubuntu 18.04.4 LTS
   - **Database**: PostgreSQL
   
   
   **What happened**:
   
   Firstly, Airflow summarizes required slots for all queued tasks. As a result, if the total number of slots required across a dag exceeds the limit, all the tasks get failed. There's a log for three tasks of weight 12 (TASK_A in the logs below), 12 (TASK_B) and 16 (TASK_C), and a pool with 24 slots: [deps_issue.log](https://github.com/apache/airflow/files/4781470/deps_issue.log)
   
   The second issue appears after the first one. The tasks get rescheduled due to missing limits. At the same time, corresponding runs report as they are finished. There's a mechanism which fails tasks in queued state if they were reported as finished. This should help with tasks stuck in queued state, but in fact it prevents tasks from being rescheduled. There's a combined log across components: [failure_issue.log](https://github.com/apache/airflow/files/4781482/failure_issue.log)
   
   
   **What you expected to happen**:
   
   Airflow executes tasks in two stages - 12+12 and 16 (or clockwise)
   
   
   **How to reproduce it**:
   
   1. Create a pool with capacity 24
   2. Create a dag with three tasks which require 12, 12 and 16 slots assigned to the pool
   3. Trigger the dag
   
   
   **Anything else we need to know**:
   
   
   All the task should be able to run simultaneously. The issue is observed on local and Celery executors.
   
   An example of a dag failing:
   ```python
   # coding: utf-8
   from airflow import DAG
   from airflow.operators.python_operator import PythonOperator
   from airflow.utils.dates import days_ago
   
   default_args = {
       'owner': 'airflow',
       'depends_on_past': False,
       'start_date': days_ago(1),
       'schedule_interval': None
   }
   
   dag = DAG(
       'THE_DAG',
       default_args=default_args,
       schedule_interval=None,
       concurrency=10,
   )
   
   
   def op(**kwargs):
       print('Hello!')
   
   
   with dag:
       PythonOperator(
           task_id='TASK_A',
           provide_context=True,
           python_callable=op,
           pool='THE_POOL',
           pool_slots=12,
       )
       PythonOperator(
           task_id='TASK_B',
           provide_context=True,
           python_callable=op,
           pool='THE_POOL',
           pool_slots=12,
       )
       PythonOperator(
           task_id='TASK_C',
           provide_context=True,
           python_callable=op,
           pool='THE_POOL',
           pool_slots=16,
       )
   ```
   


----------------------------------------------------------------
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] kaxil commented on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   Yes this will be included in 1.10.12


----------------------------------------------------------------
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] pulsar314 commented on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   There's a patch with fix of scheduler. It is based on 1.10.10 tag
   
   [0001-Fixes-treatment-of-open-slots-in-scheduler-job.patch.txt](https://github.com/apache/airflow/files/4826240/0001-Fixes-treatment-of-open-slots-in-scheduler-job.patch.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



[GitHub] [airflow] thesuperzapper edited a comment on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   @pulsar314 should we back-port this into `1.10`?
   
   I am currently just applying the patch above manually.
   
   EDIT: seems like it didn't make it into `1.10.11`: https://github.com/apache/airflow/blob/1.10.11/airflow/jobs/scheduler_job.py


----------------------------------------------------------------
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] thesuperzapper commented on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   @pulsar314 should we back-port this into `1.10`?
   
   I am currently just applying the patch above manually.


----------------------------------------------------------------
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] pulsar314 commented on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   @thesuperzapper, I also just noticed that this fix is not included into 1.10.11 :\ 


----------------------------------------------------------------
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 #9316: Tasks get failed if capacity of a pool is exceeded

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


   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] mik-laj closed issue #9316: Tasks get failed if capacity of a pool is exceeded

Posted by GitBox <gi...@apache.org>.
mik-laj closed issue #9316:
URL: https://github.com/apache/airflow/issues/9316


   


----------------------------------------------------------------
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] thesuperzapper commented on issue #9316: Tasks get failed if capacity of a pool is exceeded

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


   @potiuk can we please cherrypick this into the `1.10.12` release, as its a fairly important bug which stops tasks being scheduled, and effectively makes `pool_slots` useless.
   
   NOTE: it might need some small tweaks to work in 1.10.X


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