I like what Jarek suggested, but we should avoid using the term "Running". From 
Airflow's perspective, a Deferred task is not considered a Running task, even 
though it may be viewed differently in the user's context.

Additionally, we are currently using the term "Executing" here 
https://github.com/apache/airflow/blob/e0cd6e246c288d33f359ec2268b3d342832e9648/airflow-core/src/airflow/utils/state.py#L67

Maybe we can count Deferred and Running tasks as "Executing"? The thing that 
kinda bugs me is that "Defered" is also an IntermediateTIState here. 

On 2026/02/22 20:22:45 Natanel wrote:
> Hello Jens, I agree with everything you said, for some reason the
> "Deferred" state is not counted towards an active task, where intuitively
> it should be part of the group.
> 
> As I see it at least, all the configurations talk about *active* tasks
> (such as max_active_tasks, max_active_tis_per_dag,
> max_active_tis_per_dagrun), which I think is a quite confusing term.
> And to solve this, a clear explanation of what is an "active" task should
> be defined.
> 
> It is possible to define that an "active" task is any task which is either
> running, queued OR deferred, but this will enforce a new configuration for
> backwards compatibility, such as "count_deffered_as_active" (yet this is
> more enforcing and global approach, which we might not want), while not
> introducing too much additional complexity, as adding more parameters by
> which we schedule tasks will only make scheduling decisions harder, as more
> parameters need to be checked, which will most likely slow down each
> decision, and might slow down the scheduler.
> 
> I liked Jarek's approach, however, I think that maybe instead of
> introducing a few new params, we instead rename the current parameters,
> while keeping behavior as is, slowly deprecating the "active"
> configurations, as Jarek said, and for some time keep both the "active" and
> the "running" param, while having the "active" be an alias for "running"
> until the "active" is deprecated.
> 
> If there is a need for a param for deferred tasks, it is possible to add
> only for deferrable tasks, in order to not impact current scheduling
> decisions done by the scheduler.
> 
> I see both approaches as viable, yet I think that adding an additional
> param might introduce more complexity, and maybe should be split out of the
> regular task flow, as a deferrable task is not the same as a running task,
> I tend to lean towards the first approach, as it seems to be the simplest,
> however, the second approach might be more beneficial long-term.
> 
> Best Regards,
> Natanel
> 
> On Sun, 22 Feb 2026 at 18:43, Jarek Potiuk <[email protected]> wrote:
> 
> > +1. But I think that there are cases where people wanted to
> > **actually** use `max_*` to limit how many workers the DAG or DAG run
> > will take. So possibly we should give them such an option—for example,
> > max_running_tis_per_dag, etc.
> >
> > There is also the question of backward compatibility. I can see the
> > possibility of side effects - if that changes "suddenly" after an
> > upgrade. For example it might mean that some Dags will suddenly start
> > using far fewer workers than before and become starved.
> >
> > So - if we want to change it, I think we should deprecate "_active"
> > and possibly add two new sets of parameters with different names—but
> > naming in this case is hard (more than usual).
> >
> > J.
> >
> > On Sun, Feb 22, 2026 at 5:25 PM Pavankumar Gopidesu
> > <[email protected]> wrote:
> > >
> > > 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]
> > > >
> > > >
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: [email protected]
> > For additional commands, e-mail: [email protected]
> >
> >
> 

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to