-
Notifications
You must be signed in to change notification settings - Fork 14.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[AIRFLOW-6856] Bulk fetch paused_dag_ids #7476
[AIRFLOW-6856] Bulk fetch paused_dag_ids #7476
Conversation
4c3c638
to
5b33dc5
Compare
Codecov Report
@@ Coverage Diff @@
## master #7476 +/- ##
==========================================
- Coverage 86.81% 86.55% -0.27%
==========================================
Files 893 893
Lines 42193 42342 +149
==========================================
+ Hits 36629 36647 +18
- Misses 5564 5695 +131
Continue to review full report at Codecov.
|
.filter(DagModel.is_paused.is_(True)) | ||
.filter(DagModel.dag_id.in_(dagbag.dag_ids)) | ||
.all() | ||
) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A few lines after this we call self._process_dags, which the makes this same query again. Is it worth passing it in instead?
The other thing I'm wondering API wise is if this should be encapsulated inside DagBag (something like a dagbag.paused_dag_ids
method or accessor) - but that may not play very well with the global/long-lived DagBag object the webserver has.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch. dags parameters contain only active dags, so we don't have to check it a second time.
5b33dc5
to
ea3997b
Compare
airflow/models/dag.py
Outdated
dag_ids = set(dag_by_ids.keys()) | ||
orm_dags = session.query(DagModel)\ | ||
.options( | ||
joinedload(DagModel.tags, innerjoin=False) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This loads all the tags for all the dags we've loaded in one query, rather than needing one query for each dag. This is commonly called an n+1
query situation (which as Kamil has shown, are expensive/results in lots of extra queries)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#7477 Here is PR about DAG Sync.
ea3997b
to
ef3622f
Compare
ef3622f
to
72478de
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
59521c7
to
b1937c0
Compare
b1937c0
to
86d7abc
Compare
paused_dag_ids = ( | ||
session.query(DagModel.dag_id) | ||
.filter(DagModel.is_paused.is_(True)) | ||
.filter(DagModel.dag_id.in_(dagbag.dag_ids)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is IN query recommended here for use cases where we have 10 thousands of paused dag? Shouldn't query be broken into smaller batches?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is executed only for DAGs from one file. In one file you would have to have several thousand DAGs to cause problems. For now, I focused on the situation when we have up to 200 DAGs. If we want to support several thousand DAGs in one file, we need to introduce much more optimization and this one would not change anything.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry for trouble, I missed DAGs from one file part earlier I thought we are scanning across folder
I created the following DAG file:
and I used the following code to test performance.
As a result, I obtained the following values
Before:
Query count: 442
Average time: 1182.187 ms
After:
Query count: 297
Average time: 769.421 ms
Diff:
Query count: -145 (-32%)
Average time: -413 ms (-34%)
When we have 200 DAGS with 5 tasks each in one file:
Before:
Query count: 1792
Average took 4505.891 ms
After:
Query count: 1197
Average took 3335.856 ms
Thanks for support to @evgenyshulman from Databand!
Issue link: AIRFLOW-6856
Make sure to mark the boxes below before creating PR: [x]
[AIRFLOW-NNNN]
. AIRFLOW-NNNN = JIRA ID** For document-only changes commit message can start with
[AIRFLOW-XXXX]
.In case of fundamental code change, Airflow Improvement Proposal (AIP) is needed.
In case of a new dependency, check compliance with the ASF 3rd Party License Policy.
In case of backwards incompatible changes please leave a note in UPDATING.md.
Read the Pull Request Guidelines for more information.