You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "james-seymour-cubiko (via GitHub)" <gi...@apache.org> on 2023/02/08 07:55:36 UTC

[GitHub] [airflow] james-seymour-cubiko opened a new issue, #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool

james-seymour-cubiko opened a new issue, #29416:
URL: https://github.com/apache/airflow/issues/29416

   ### Description
   
   Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool - not sure what the best way of implementing this is, but currently my very hacky solution is to patch the `airflow.models.pool.Pool.slots_stats` method to include deferred tasks as running in each pool.
   
   ### Use case/motivation
   
   The prototypical usecase here is using Airflow to limit the number of concurrent queries executing against a database while keeping the benefit of waiting for those queries to complete on a triggerer (where a proxy is used to execute queries instead of a direct connection to the db)
   
   In our case, we use Airflow to orchestrate an Azure Data Factory that executes queries against a database and moves the resulting data. 
   
   We have an airflow task trigger a single pipeline run in that data factory, which then defers and waits for that pipeline run to complete in the triggerer (for efficiency) before continuing the dag run. 
   
   However, we have ~100 tasks that all execute a pipeline run on the same factory - ideally we would execute all of these pipelines concurrently, but the database is quickly overwhelmed by that many queries at the same time, resulting in timeouts. Therefore, the next best option is to limit the concurrency of those queries with a task pool in Airflow.
   
   This _can_ currently be achieved with Airflow's task pools, but only if we keep each of those tasks in the running state while waiting each query to complete (as deferred tasks do not occupy slots in the task pool). Otherwise, if we defer the tasks while waiting, then we lose the concurrency limits of the pool, as all ~100 tasks are free to defer at the same time, so its currently an either / or solution.
   
   I am aware that in this specific case that ADF does support a maximum pipeline run concurrency setting, which is a much quicker way to solve this problem, but we have other extraction tools that we can't rely on to limit concurrency in this way, and I thought I would just throw this idea out here anyway in case others might find it helpful :)
   
   ### Related issues
   
   Somewhat related - https://github.com/apache/airflow/issues/15082
   
   ### Are you willing to submit a 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: commits-unsubscribe@airflow.apache.org.apache.org

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


Re: [I] Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk closed issue #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool
URL: https://github.com/apache/airflow/issues/29416


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


Re: [I] Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool [airflow]

Posted by "Usiel (via GitHub)" <gi...@apache.org>.
Usiel commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1805031834

   I think this issue is solved by https://github.com/apache/airflow/pull/32709 already.


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


Re: [I] Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1805293832

   Ah yes, indeed.


-- 
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] tanelk commented on issue #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool

Posted by "tanelk (via GitHub)" <gi...@apache.org>.
tanelk commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1423685677

   There is one more thing to keep in mind.
   
   The lifecycle of a deferrable task is something like this:
   none -> scheduled -> queued -> running -> deferred -> scheduled -> queued -> running -> success
   Currently it only consumes pool slots in queued and running states and you'd like to add deferred to that list. In that chain there still is a scheduled (the second one), that splits the execution into two parts. Handling that could be little more work than just updating the `Pool.slots_stats`. `TI.next_method` could be used to distinguish the two scheduled states but that does not jibe with scheduler.


-- 
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] pvsmpraveen commented on issue #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool

Posted by "pvsmpraveen (via GitHub)" <gi...@apache.org>.
pvsmpraveen commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1565042115

   Similar usecase, but with Sensor mode='reschedule'


-- 
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 #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool

Posted by "boring-cyborg[bot] (via GitHub)" <gi...@apache.org>.
boring-cyborg[bot] commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1422179024

   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 #29416: Optionally allow a task pool to count tasks in the 'deferred' state as occupying slots in that pool

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on issue #29416:
URL: https://github.com/apache/airflow/issues/29416#issuecomment-1422255026

   Yes. The small problem is that this must be a global feature due to the way how pools work for now and it would be a potentially breaking behaviour and one that has potential to dead-lock in some circumstances. 
   
   But I tihnk we could likely do it in the way that Pool could have a "include_deferred" flag that could be false by default and I think the query in scheduler could be efficiently using it to include/exclude deferred tasks. That would be a nice "feature" - but it has a few consequences across the board (modifying CLI, UI, etc.).
   
   Overall it's a nice feature for someone who would like to get more insights on some Airflow internals.


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