Hi Jens, Thanks for starting this discussion. I agree that we should update how these tasks are counted.
I previously started a PR[1] to include deferred tasks in max_active_tasks, but I was sidetracked by other priorities. As you noted, this change needs to encompass not only max_active_tasks but also the other parameters you described. [1]: https://github.com/apache/airflow/pull/41560 Regards, Pavan On Sun, Feb 22, 2026 at 12:43 PM constance.astronomer.io via dev < [email protected]> wrote: > Agreed. In my opinion, the only time we should not be counting deferred > tasks are for configurations that control worker slots, like number of > tasks that run concurrently on a celery worker, since tasks in a deferred > state are not running on a worker (although you can argue that a triggerer > is a special kind of worker but I digress). > > For the examples you’ve listed, deferred tasks should be part of the > equation since the task IS running, just not in a traditional worker. > > Thanks for bringing this up! This has been bothering me for awhile. > > Constance > > > On Feb 22, 2026, at 4:18 AM, Jens Scheffler <[email protected]> wrote: > > > > Hi There! > > > > TLDR: In fix PR https://github.com/apache/airflow/pull/61769 we came to > the point that it seems today in Airflow Core the "Deferred" state seems to > be counted inconsistently. I would propose to consistently count "Deferred" > into the counts of "Running". > > > > Details: > > > > * In Pools for a longer time (since PR > > https://github.com/apache/airflow/pull/32709) it is possible to > > decide whether tasks in deferred state are counted into pool > > allocation or not. > > * Before that Deferred were not counted into, which caused tasks being > > in deferred potentially overwhelm backends which defesated the > > purpose of pools > > * Recently it was also seen that other limits we usually have on Dags > > defined as following do not consistently include deferred into limits. > > o max_active_tasks - `The number of task instances allowed to run > > concurrently` > > o max_active_tis_per_dag - `When set, a task will be able to limit > > the concurrent runs across logical_dates.` > > o max_active_tis_per_dagrun - `When set, a task will be able to > > limit the concurrent task instances per Dag run.` > > * This means at the moment defining a task as async/deferred escapes > > the limits > > > > Code references: > > > > * Counting tasks in Scheduler on main: > > > https://github.com/apache/airflow/blob/main/airflow-core/src/airflow/jobs/scheduler_job_runner.py#L190 > > * EXECUTION_STATES used for counting: > > > https://github.com/apache/airflow/blob/main/airflow-core/src/airflow/ti_deps/dependencies_states.py#L21 > > o Here "Deferred" is missing! > > > > Alternatives that I see: > > > > * Fix it in Scheduler consistently that limits are applied counting > > Deferred always in > > * There might be a historic reason that Deferred is not counting in - > > then a proper documentation would be needed - but I'd assume this > > un-likely > > * There are different opinions - then the behavior might need to be > > configurable. (But personally I can not see a reason for having > > deferred escaping the limits defined) > > > > Jens > > --------------------------------------------------------------------- > To unsubscribe, e-mail: [email protected] > For additional commands, e-mail: [email protected] > >
