Hi,

oh, two large / related discussions emails. A lot of arguments and details. I fear when a lot of people respond, the solution space fragments. Can you write an AIP document (Airflow Improvement Proposal) with a future design and thoughts? Then this can be individually commented and agreed.

In some points I agree that priorities are hard to be estimated (except if absolute strategy is used) which we compensate with ranges in batches of 100. As we also schedule a lot of tasks we also make use of absolute priority as this saves also a lot of CPU in scheduler. I assume mainly people also do not have transparency "why" the tasks are not scheduled in most cases and it can be because scheduler is overwhelmed or other side effects of other tasks steal priority. I was also questioning myself why "absolute" strategy is not the default.

Nevertheless both pools and priorities have a reason. Mostly a priority per Dag might be sufficient. But changing this scheme today would be rates like a breaking change in the core of the Airflow model. So from the release model it would be something we would need to make an Airflow 4.0 for.

> The whole global constraint means we MUST each time sort the entire task instance table and be ready to scan it in its entirety.

Here I dis-agree. The sorting is made on DB level and it is the core competence of the DB to provide a sorted list. And it is important that is is possible even when you have a large backlog of work that you can prioritize important work (not putting it at the end). We require this and I assume many other require this as well. The complexity comes together with Dag and Task limitations and Pools which throttle workload.

So I'd be happy about a concept and also where not documented in this concept we might need to map the (historically grown) requirements against this. It might be valid (and this is to be discussed) if some historic requirement is not valid anymore which relaxes the constraints and opens the door for new options.

Jens

On 28.09.25 19:22, asquator wrote:
Hello,

I've come to a conclusion that the task priorities we have today are flawed, 
vaguely defined and very weak in nature, their job is hard to predict and 
overall they make the scheduler difficult to improve.

I've been involved for at least half a year now looking for a better scheduler 
strategy to avoid starvation in the critical section and increase the overall 
queueing throughput. Different strategies were examined, including window 
functions (#53492) and stored procedures (#55537). They both have their flaws, 
though the second one gives a blazingly fast throughput- the main problem with 
it is coupling to the DB (as a stored procedure has to be maintained per SQL 
flavor) - unfortunately, every attempt to rewrite it as Python code bumped into 
the priority weights we have to globally maintain per every single task 
instance in the QUEUED state.

The whole global constraint means we MUST each time sort the entire task 
instance table and be ready to scan it in its entirety. We can't concurrently 
examine different DAG runs and queue tasks independently. Does it pay well? No, 
because in case we find no tasks to queue in the CS, we just continue looking 
for less prioritized tasks and queue them if we find some.

Moreover, maintaining priority weights per task is VERY difficult in multi-tenant clusters. With 
weight rules (even the default ones except ABSOLUTE), we can barely know how prioritized a certain 
task will be in runtime. When different users start messing up with priorities, we start 
experiencing noisy neighbors issues and users "stealing" the entire "broadband".

Turns out priorities don't really work when we need them:
https://github.com/apache/airflow/issues/26933

I had some talks with @Nataneljpwd and Daniel S., and they acknowledge the 
issues we have today with priorities:

- They're vaguely defined and unpredictable
- They pose a weak limit
- They don't do what they're expected to do due to implementation complexity
- They make the scheduler nasty to change
- They're hard to maintain in big clusters
- They don't fit the prioritization use case well enough

I think priorities have to change. If we MUST retain the current parameter 
named priority_weight​, it can remain. We can make it even weaker by demanding 
even less of this parameter, which may benefit us in fixing the starvation bug.
On the other hand, we DO want some notion of priorities in airflow, and we can 
define the new concept and implement it, while making the current 
priority_weight even weaker.

We had some thoughts regarding the desired priorities:

- As we think in terms of "data workflows", a.k.a DAGs, priority per DAG may be 
fit.
- Priority weights per pool.
- Doing 1 or 2 and introducing a new object representing a logical "tenant", so 
the priorites can be defined per a single tenant.

What do you think?

Reply via email to