Re: [I] DAG Runs from a single DAG can prevent scheduler from seeing other DAG's runs [airflow]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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]
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