Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-06-10 Thread via GitHub


amoghrajesh commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2961427174

   Removing the milestone as we do not have a clear repro environment for this.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-06-05 Thread via GitHub


dstandish commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2945364573

   Yeah Collin I think we need clarification on the repro scenario / under what 
conditions starvation occurs.  
   
   Let me go through your post carefully
   
   > Two DAGs each receive a large batch of DAG Runs. The number of runs for 
each DAG exceeds max_dagruns_per_loop_to_schedule. 
   What state are the "received" in?  I assume you mean triggered via API?
   > Each DAG run is very short, shorter than the heartrate of this Airflow 
deployment. 
   What is the signifigance of the runs being very short? What do you think 
that has to do with this?
   > Both DAGs have a max_active_runs that is far less than dagruns_per_loop.
   Why?  So that we can be confident that the scheduler _should_ fetch some of 
these runs in the query?
   > So: max_active_runs < max_dagruns_per_loop_to_schedule < number of queued 
DAG runs.
   > Each scheduler loop, there are a very small number of DAG Run "slots" for 
the first DAG, so the check coalesce(running_drs.c.num_running, text("0")) < 
coalesce(Backfill.max_active_runs, DagModel.max_active_runs), does not apply. 
But then all the DAG runs that are considered are from the first DAG. So Second 
DAG effectively has to wait for nearly all of First DAG's runs to complete 
before any of its runs are moved from queued to running.
   
   Could it be that what you were observing had to do with last scheduling 
decision not getting updated because tasks were backed up somehow?
   
   I guess either way, this seems like a very rare edge case scenario and, if 
two people tried and failed it, not sure it's worth contiuning to try without 
new information.
   
   


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-06-03 Thread via GitHub


amoghrajesh commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2938785070

   Reducing to medium priority as we cannot repro it easily.
   
   


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-06-01 Thread via GitHub


vatsrahul1001 commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2928864529

   I tried reproducing this again, but I wasn't able to.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-18 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2889642970

   I’ve reopened #45636 to track the task instance issue.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-16 Thread via GitHub


Asquator commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2887162602

   > [@Asquator](https://github.com/Asquator) What’s your thought on how 
windowing can be implemented here?
   
   I implemented a minimal query that considers _Backfills_ and _DAGruns_ to 
find the greatest number of each DAG that can be scheduled: 
https://github.com/apache/airflow/pull/50692
   I'm not sure if the windowing logic belongs to `DagRun`, it's more of a 
scheduler's responsibility.
   
   On the whole I really liked the solution @uranusjr proposed, at least for 
_DAGruns_ where there's no strict priority ordering - just update 
`last_scheduling_decision` every time the model is touched. Looks like it's 
original intention was for _TaskInstances_, but I don't see if it's ever 
updated after initialization. It can be also extended to _TaskInstances_ if 
we're able to change the behavior of priority weights.
   
   Also, I think the parallel discussion about starving  _TaskInstances_ should 
be reopened since these are two different bugs, and the fixes may be different.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-16 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2886094860

   I have a weird finding. I set up the dags similar to 
https://github.com/apache/airflow/issues/49508#issuecomment-2834521949 above, 
set up a `special_pool` with 2 slots. 16 runs are created against `scheduler1`, 
for every day from 2023-02-01 to 2023-02-16. Everything is as expected so far. 
Runs 1st to 8th are running, 9th to 16th are scheduled.
   
   Now I trigger a manual run on `scheduler2` (which uses `special_pool` 
instead of the default). The scheduler shows one task has been triggered. But 
if you look at the logs closely, the supervisor wants to trigger the 9th ti for 
`scheduler1` instead!
   
   ```
   [supervisor] 
msg=StartupDetails(ti=TaskInstance(id=UUID('0196d838-9ad3-7c16-a551-f9997f0b5960'),
 task_id='sleep_it1', dag_id='scheduler1', 
run_id='scheduled__2023-02-09T00:00:00+00:00', try_number=1, map_index=-1, 
pool_slots=1, queue='default', priority_weight=1, executor_config={}, 
parent_context_carrier={}, context_carrier=None, queued_dttm=None), 
dag_rel_path='schestuck.py', bundle_info=BundleInfo(name='dags-folder', 
version=None), requests_fd=39, start_date=datetime.datetime(2025, 5, 16, 8, 32, 
30, 713298, tzinfo=datetime.timezone.utc), 
ti_context=TIRunContext(dag_run=DagRun(dag_id='scheduler1', 
run_id='scheduled__2023-02-09T00:00:00+00:00', 
logical_date=datetime.datetime(2023, 2, 9, 0, 0, tzinfo=TzInfo(UTC)), 
data_interval_start=datetime.datetime(2023, 2, 9, 0, 0, tzinfo=TzInfo(UTC)), 
data_interval_end=datetime.datetime(2023, 2, 9, 0, 0, tzinfo=TzInfo(UTC)), 
run_after=datetime.datetime(2023, 2, 9, 0, 0, tzinfo=TzInfo(UTC)), 
start_date=datetime.datetime(2025, 5, 16, 8, 31, 50,
  488281, tzinfo=TzInfo(UTC)), end_date=None, clear_number=0, 
run_type=, conf={}, 
consumed_asset_events=[]), task_reschedule_count=0, max_tries=0, variables=[], 
connections=[], upstream_map_indexes={}, next_method=None, next_kwargs=None, 
xcom_keys_to_clear=[], should_retry=False), type='StartupDetails')
   ```
   
   I think this is due to how LocalExecutor manages task information with a 
`multiprocessing.Queue`. When a pool reaches limit, somehow _one more_ message 
gets pushed into `activity_queue`, and it clogs up everything behind it until 
the executor can finally process it after all previous tis are done.
   
   Also note that this is against 3.0. Implementation around this changed a bit 
between 2 and 3 (due to task sdk), so it’s possible we have two different bugs 
in different versions. It’s also possible the culprit is the same in 2 since 
the `activity_queue` usage did not change that much.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-16 Thread via GitHub


Nataneljpwd commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2877636652

I think I agree with what @Asquator mentioned, using windowing will solve 
this issue and fix most of the edge cases where dags are starved along with 
tasks which are starved or stuck on queued, Windowing could be a very good 
approach that will be able to go over all edge cases by each time selecting 
`max_tis` tasks to schedule if possible and if not, select the largest amount 
of tasks to schedule, however it might increase the time it takes to run the 
query, however, might not necessarily  increase the duration in the critical 
section as we will send less queries to the database when encountering starving 
or tasks that we cannot schedule.
   
   I think that if written in sql (a rough draft, might contain mistakes) , the 
windowing will look something like this:
   
   ```sql
   select *,
   
   sum(task_instance.pool_slots) over (
 partition by pool order by (-priority_weight /* to get tasks by priority 
*/, updated_at /* to favor tasks which are longer in queue */, map_index /* to 
allow for mapped tasks as well to run in order */)
 ) as slots_taken
   
   count(task_instance.id) over (
 partition by dag_run.id order by(/*same as before*/)
   ) as tasks_per_dag,
   
   (select count(task_instance.id) where task_instance.state="running" and 
dag_run.dag_id = task_instance.dag_id) as tasks_per_dag,
   
   where slots_taken <= pool.slots and
   tasks_per_dag <= dag.max_active_tasks - running_tasks_per_dag and
   task_instance.state="SCHEDULED" /*and 
   max_ti_per_task_per_dagrun <= dag.max_ti_per_task_per_dagrun -- needs to be 
added to the database
   */
   
   from dag join dag_run on dag_id
   join task_instance on dag_id
   join slot_pool as pool on task_instance.pool = pool.pool
   
   group by dag_run
   
   limit {{max_tis}}
   ```
   
   This query is untested, and heavily inspired by the query @Asquator wrote in 
#49160 with small calculation fixes.
   For this to work, a configuration needs to be added to the airflow db to the 
`dag` model called `max_active_tis_per_dagrun` which exists after #29094 and 
does not exist in the database model, after that the query can be complete.
   
   Another quicker solution which could work is to change the prioritization to 
be probability based rather than strictly larger first, for example, get all 
priorities and give top N priorities at least 1 slot and then by relative 
weight distribute the task instances or give the top priority x slots, the next 
in line x / 2 + leftover and so on.
   
   The second solution could be implemented using counts and windows over 
priority as well, which will not make the query too heavy however might 
increase the query time, we could also think of a simple mathematical algorithm 
to do it for any fixed amount of priorities and implement it in sql while 
having almost no impact on performance.
   
   Just changing the sql query for the scheduler might work for most cases, 
however there might still be cases where starvation occues, the second solution 
should fix starvartion of dags and tasks better but it might not maximize the 
performance of the scheduled to queued loop.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-12 Thread via GitHub


nevcohen commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2875157236

   > If anyone can try this out https://github.com/apache/airflow/pull/50379
   
   The change you suggested affects the transition of dagruns from queued to 
running, and doesn't address the problem that (from what I understand) was 
raised here and that I wrote about 
https://github.com/apache/airflow/issues/45636.
   
   The problem is tasks whose dagrun is already running but the task doesn't 
progress from queued to running.
   
   


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-09 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2865279081

   If anyone can try this out https://github.com/apache/airflow/pull/50379


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-08 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2864959991

   I don’t think multiple schedulers would help since all those schedulers 
would pull runs with the same query in `get_queued_dag_runs_to_set_running`, 
and once you hit the point of starvation, they all get stuck on the same runs 
over and over, no matter how many schedulers you have. For multiple schedulers 
to help, we need to introduce additional discrimination logic between 
schedulers so they look at different dag runs to have a possibility to untangle 
the situation.
   
   Another thing I’m thinking now is, maybe we should just update 
`last_scheduling_decision` when a dag run is selected in the queued state? 
Currently we only update the field after the run enters the running state, so 
when no more runs can enter running, the flag never changes, and the query is 
stuck selecting the same runs over and over and not making progress. I think 
this can work…? Not sure. Anyone interested to help verify this?


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-08 Thread via GitHub


eladkal commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2864013041

   > Is it helping in the situation if you have multiple schedulers, e.g. 3 
instances?
   
   It does not. Had 8 schedulers.
   This was my first try as well.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-08 Thread via GitHub


jscheffl commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2863921228

   As we have currently (and especially this week) also massive scheduling 
problems... I can say that with Celery we are atm able to make the heartbeat 
really slow... maybe another problem but the example to re-produce we could do 
easily atm.
   
   But I am coming back to this: Is it helping in the situation if you have 
multiple schedulers, e.g. 3 instances? Then even if the loop of one is done and 
until it gets back, some other scheduler in parallel will most probably 
come-along (no gurarantee though) and maybe exclude the one DAG that is hiding 
the other runs. Would this be a workaround?


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-08 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2862331087

   @Asquator What’s your thought on how windowing can be implemented here?


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-05-01 Thread via GitHub


Asquator commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2844711646

   Looks convincing. According to OP, the second DAG's runs are stuck at 
_queued_ and not getting scheduled, so it shouldn't be related to 
`get_running_dag_runs_to_examine` or pools or any issues with the critical 
section, because _queued -> running_ transition for DAG runs happens much 
earlier than we get to any scheduling decisions for tasks (like creating TIs or 
scheduling them in CS). The whole logic for transitioning DAG runs to _running_ 
is in `SchedulerJobRunner._start_queued_dagruns` and 
`DagRun.get_queued_dag_runs_to_set_running`, and the described behaviour is a 
consequence of the optimistic scheduling strategy used across Airflow not being 
able to handle a variety of edge cases. 
   
   We first pull at most `DEFAULT_DAGRUNS_TO_EXAMINE` DAG runs whose DAGs are 
not yet maxed out:
   
https://github.com/apache/airflow/blob/e4957ff3827e0aea0465026023dd58288c5b1299/airflow-core/src/airflow/models/dagrun.py#L602C13-L611C14
   
   And then go over each one of them increasing the currently scheduled DAGrun 
count and dropping once we can't stuff any more runs:
   
https://github.com/apache/airflow/blob/65be581ac7715d3765af4a3b99faea26e5da55fc/airflow-core/src/airflow/jobs/scheduler_job_runner.py#L1740C9-L1740C33
   
https://github.com/apache/airflow/blob/65be581ac7715d3765af4a3b99faea26e5da55fc/airflow-core/src/airflow/jobs/scheduler_job_runner.py#L1762C13-L1773C29
   
   
   It's indeed the same as https://github.com/apache/airflow/issues/45636, but 
now with DAG runs instead of tasks. If the original issue can be efficiently 
solved with windowing, I guess this one can also benefit from it, like:
   
   1. Window partitioned by DAG
   2. For each DAG compute the number of runs
   3. Stuff new runs as far as there are free slots (per DAG)
   4. Put a limit on the result (which can be far less restrictive, as we 
delegate more work to SQL and the cost of a query won't go up as fast as 
throughput will).
   
   As windowing may introduce additional overhead which sometimes won't result 
in higher throughput, assuming there aren't so many DAG runs in the system, I'm 
looking for a compromise between the optimistic and the windowing (pessimistic) 
approaches. It's possible to have it configurable, though it will make the 
system harder and harder to tweak to everyone's needs, which is not desirable. 
I hope the windowing can be done in such a way that will be negligible for the 
query time, by the help of correct indexing, or completely changing some core 
concurrency parameters, like introducing `DEFAULT_DAGS_TO_EXAMINE` in favor of 
`DEFAULT_DAGRUNS_TO_EXAMINE`, where the new parameter limits the number of 
windows in the query. Some research is needed to find out. 


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-28 Thread via GitHub


jscheffl commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2836466232

   We also had such issue (had a talk about this in 
https://airflowsummit.org/sessions/2024/how-we-tuned-our-airflow-to-make-1-2-million-dag-runs-per-day/)
 and I _think_ my colleguage @AutomationDev85 supplied a fix to Airflow 2.6 or 
2.7... we our problem in this area were gone.
   
   But one important remark: We ensured that the amount of parallel DAG runs is 
limited. We triggered new DAG runs in state "queued" and not "running", then 
the scheduler saves efforts in scheduling "not ready" runs. And was only adding 
"running" state at a point when sufficient DAGs have been worked-off. Can you 
try this?


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-28 Thread via GitHub


eladkal commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2835131897

   > Aha, yes. That must be the missing ingredient that made it "hard" to 
reproduce. Still don't fully understand why each DAG Run in DAG 1 don't get 
marked with scheduling_decision and then DAG 2 is checked. It seems like the 
scheduler should be able to check all runs faster than 30 minutes.
   
   Because when we read records from the database in the scheduler critical 
section we read as bulk. We don't have extra logic as we want this part to be 
as fast as possible.  The logic comes after and you apply it on the bulk 
records you read (`AIRFLOW__SCHEDULER__MAX_TIS_PER_QUERY`)


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-28 Thread via GitHub


collinmcnulty commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2835102497

   Aha, yes. That must be the missing ingredient that made it "hard" to 
reproduce. Still don't fully understand why each DAG Run in DAG 1 don't get 
marked with scheduling_decision and then DAG 2 is checked. It seems like the 
scheduler should be able to check all runs faster than 30 minutes.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-28 Thread via GitHub


eladkal commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2834521949

   It seems that starvation can happen. Also reported in 
https://github.com/apache/airflow/issues/45636
   
   Given 2 pools `default_pool` and `special_pool` with 2 DAGs:
   
   ```
   from datetime import datetime
   from airflow import DAG
   from airflow.decorators import task
   import time
   
   default_args = {
   'owner': 'airflow',
   'start_date': datetime(2023, 2, 1)
   }
   
   with DAG('scheduler1', schedule="@daily", catchup=True, 
default_args=default_args):
   
   @task()
   def sleep_it1():
   time.sleep(3600) # 1 hour
   sleep_it1()
   
   with DAG('scheduler2', schedule=None, catchup=False, 
default_args=default_args):
   @task(pool="special_pool")
   def sleep_it2():
   time.sleep(60) # 1 min
   
   sleep_it2()
   ```
   
   
   The scheduler1 dag creates many dagruns as it has catchup. Once those runs 
are created there is large "backlog" of tasks to complete, create a run in 
scheduler2. The task sleep_it2 will not be executed till all runs of sleep_it1 
are finished. That is though  in each loop of the scheduler it can not really 
schedule tasks of sleep_it1 but can schedule sleep_it2.
   I waited for more than 30 min and sleep_it2 did not get scheduled though it 
can be.
   
   https://github.com/user-attachments/assets/caab1d8e-8220-400f-b7b6-d6803e31457c";
 />
   
   This happens both in Airflow 2 and Airflow 3.


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-22 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2820844220

   I tried to reduce this to a reasonable minimal scenario. This is against 
main since we did not notice any significant logic changes in the area.
   
   * Two dags `1` and `2` both have one single task. Both dags set 
`max_active_runs=1`. The task simply does `sleep(5)`.
   * Set `AIRFLOW__SCHEDULER__MAX_DAGRUNS_PER_LOOP_TO_SCHEDULE` to `1`.
   * Trigger 10 runs for `1`, and then 10 for `2`, both via API (I used a 
Python script with `requests` so the ordering is deterministic).
   
   After triggering, I can observe the scheduler actually alternates between 
the two dags. It picks up a run from the first dag, and then a run from the 
second, the first, the second, and so on. Interetingly, I _did not_ see any 
logs on `max_active_runs` at all; the scheduler seems to just do it out of the 
box.
   
   So I think we’re missing something. Not sure what…


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-21 Thread via GitHub


uranusjr commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2819048593

   The weird thing to me is we _seems_ to have a mechanism for this
   
   
https://github.com/apache/airflow/blob/e4957ff3827e0aea0465026023dd58288c5b1299/airflow-core/src/airflow/models/dagrun.py#L522-L539
   
   This should prioritize runs we have not touched (nulls first), or touched 
the least recently. So even if a run is not seen in an iteration, runs before 
it (and thus touched) should be deprioritized until all other runs were also 
touched, and so on, so each run should get a chance at some point. Maybe the 
timestamp update is not working as expected?


-- 
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-unsubscr...@airflow.apache.org

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



Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]

2025-04-21 Thread via GitHub


collinmcnulty commented on issue #49508:
URL: https://github.com/apache/airflow/issues/49508#issuecomment-2818976612

   The workaround we're using is to set `max_dagruns_per_loop_to_schedule` to a 
value greater than the largest number of Runs we'll get for any individual DAG.


-- 
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-unsubscr...@airflow.apache.org

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