-
Notifications
You must be signed in to change notification settings - Fork 14.5k
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-2694] Declare permissions in DAG definition #4642
Conversation
This PR adds support for declaratively assigning DAG-level permissions to a role via the `DAG.__init__()` method. When the DAG definition is evaluated and the `access_control` argument is supplied, we update the permissions on the ViewMenu associated with this DAG according to the following rules: - If the role does not exist, we raise an exception. - If the role exists, we ensure that it has the specified set of permissions on the DAG - If any other permissions exist for the DAG that are not specified in `access_control`, we revoke them
a724abd
to
bbfc3c7
Compare
hey @astahlman , will take a look next week. |
Codecov Report
@@ Coverage Diff @@
## master #4642 +/- ##
==========================================
- Coverage 74.3% 74.26% -0.04%
==========================================
Files 426 427 +1
Lines 27867 27942 +75
==========================================
+ Hits 20706 20751 +45
- Misses 7161 7191 +30
Continue to review full report at Codecov.
|
airflow/models/__init__.py
Outdated
:type access_control: dict | ||
|
||
""" | ||
sm = cached_appbuilder().sm |
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 may be a little concern as it requires starting the webserver when Airflow starts the scheduler.
Two ideas could help solve these cases:
- move all FAB models(https://github.com/dpgaspar/Flask-AppBuilder/blob/1e900bba85452de6d988f7da191f9a26fec62226/flask_appbuilder/security/sqla/models.py) into Airflow code base(cc @mistercrunch who brought up this idea previously). This helps Airflow not relied on FAB that much.
- Created an API endpoint in web server for creating permission / delete permission purpose(https://issues.apache.org/jira/browse/AIRFLOW-2693). In this way, the Airflow web server handles all the FAB permission actions. And scheduler just delegates the permission handling to the webserver.
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.
@feng-tao Yeah, I had the same thought - I really don't like doing this in the __init__
method.
I see that we are already doing some permission handling in the /refresh
endpoint - maybe this would be a natural place to invoke dag.set_dag_permissions()
?
Line 1682 in 81f7c47
appbuilder.sm.sync_perm_for_dag(dag_id) |
We could have similar logic in the sync_perm
CLI command: for each DAG in the DagBag, call dag.set_dag_permissions()
Line 1634 in 81f7c47
def sync_perm(args): |
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.
I think extending the /refresh
endpoint should work. We need to pass the access_control
info from scheduler to webserver endpoint during the DAG _init_
constructor. I am not so sure how to put the set_dag_permissions
function inside sync_perms
cli though.
I think we couldn't get the dag access_control information until scheduler parses the dag information. And we need to persist these infomation somewhere in the db. Then the cli knows how to persist the permission with roles mapping information, right? In this case, I don't know how the CLI could figure how to call the sync_perm_for_dag
.
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.
sync offline, and @astahlman will move the logic into the sync_perm
cli. This is to avoid putting extra latency into scheduler loop.
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.
Updated - I've moved this logic from the DAG class to the AirflowSecurityManager and triggered it from the CLI sync_perm
command as well as the /refresh
and /refresh_all
endpoints. This way is much better I think - thanks for the feedback.
Move the DAG-level permission syncing logic into `AirflowSecurityManager.sync_perm_for_dag`, and trigger this method from the CLI's `sync_perm` command and from the `/refresh` and `/refresh_all` web endpoints.
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, thanks. Let's wait for a few days in case @jgao54 and @mistercrunch have further comment.
Once this pr is merged, I think we could declare Airflow DAG level support is done :) |
Will merge this pr now. If there are any issues brought up by others, we could fix in a later pr. But this helps to make Airflow DAG level access feature complete. Thanks @astahlman . |
Is there a reason we don't (additionally) sync the DAG parms at webserver start up? I had to dig in to the code to find why my dag perms weren't being written to the DB - I assumed that if we changed them then the webserver would pick them up when the scheduler procesed the DAG. Something about the current approach doesn't quite sit with me (I think the scheduler should be doing the updating automatically) but I don't really want the scheduler to have to know about the FAB tables. How about something like this:
Dunno. This was unexpected is all. |
Also (not sure it relates to this PR or another one) but what happens if I specify a |
@ashb , we do sync perms in https://github.com/apache/airflow/blob/master/airflow/www/app.py#L181 .Which roles with what permissions are you observing not written to the dB? Yes, as you mentioned scheduler can't see or shouldn't see the FAB tables ATM(Airflow doesn't even have the FAB table code in the model file). Hence we use this approach which works at Lyft new use case:
I am not sure why we need additional columns in the DAG table. So far Andrew's pr works well internally. I am happy to help out more if you provide more details. |
@ashb thanks for raising - I'm also not totally satisfied with the current implementation. What we have has worked so far, but we do have the same problem of DAG permissions getting out of sync after a DAG's What do you think @feng-tao? |
There are so many different discussions today. Here is another long list(#4569). @astahlman , ideally we should, and that is part of original plan / discussion for DAG level access(#3197). We(Me, Max, Joy) discussed multiple approaches for scheduler auto updated within that pr:
We(that pr) end up with create a cli for periodically sync_perm. Not sure how @ashb's approach would work by adding two columns in DAG table given there are no Airflow models defined what are "roles"(only exist in FAB models). The role column I assume should be some FK to the ab_roles table. Not sure what I described makes sense to you. |
* [AIRFLOW-2694] Declare permissions in DAG definition This PR adds support for declaratively assigning DAG-level permissions to a role via the `DAG.__init__()` method. When the DAG definition is evaluated and the `access_control` argument is supplied, we update the permissions on the ViewMenu associated with this DAG according to the following rules: - If the role does not exist, we raise an exception. - If the role exists, we ensure that it has the specified set of permissions on the DAG - If any other permissions exist for the DAG that are not specified in `access_control`, we revoke them * Move RBAC constants to break circular dependency * Add license header * Sync DAG permissions via CLI and /refresh* endpoints Move the DAG-level permission syncing logic into `AirflowSecurityManager.sync_perm_for_dag`, and trigger this method from the CLI's `sync_perm` command and from the `/refresh` and `/refresh_all` web endpoints. * Default access_control to None
Hi, @feng-tao and @astahlman : Thanks for this commit and I wonder why this function didn't implement on rbac views ? I'm using RBAC view and have 200+ DAGs, which made my group permission explosion.. |
@icyfox-bupt Airflow's "release workflow" is that all commits are made to master and then cherry-picked back on to the release branch ( The reason that isn't there yet is that this is a big change and the cherry-pick/conflict is difficult. If you want to attempt back-porting this feature to the release branch that would be welcomed. I can help you through what that process would involve if you are volunteering to do the work :) |
What a quick response! Thanks @ashb , I will try it on my project then consider to contribute~! |
@ashb didn't know this pr is not in preview release....I think this one is very important for DAG level access. @icyfox-bupt I am currently writing a blog post on DAG level access(long due....) and share how Lyft uses it, hopefully it will help clarify some of the questions you have. The blog should be out in the next 2-3 weeks. |
@feng-tao Yeah, I've been wanting to backport it when I tried (somewhere around 1.10.2? .3?) it was full of conflicts, and I haven't had time to look again -- I don't know if it's better or worse now. |
* [AIRFLOW-2694] Declare permissions in DAG definition This PR adds support for declaratively assigning DAG-level permissions to a role via the `DAG.__init__()` method. When the DAG definition is evaluated and the `access_control` argument is supplied, we update the permissions on the ViewMenu associated with this DAG according to the following rules: - If the role does not exist, we raise an exception. - If the role exists, we ensure that it has the specified set of permissions on the DAG - If any other permissions exist for the DAG that are not specified in `access_control`, we revoke them * Move RBAC constants to break circular dependency * Add license header * Sync DAG permissions via CLI and /refresh* endpoints Move the DAG-level permission syncing logic into `AirflowSecurityManager.sync_perm_for_dag`, and trigger this method from the CLI's `sync_perm` command and from the `/refresh` and `/refresh_all` web endpoints. * Default access_control to None (cherry picked from commit 5275a8f)
* [AIRFLOW-2694] Declare permissions in DAG definition This PR adds support for declaratively assigning DAG-level permissions to a role via the `DAG.__init__()` method. When the DAG definition is evaluated and the `access_control` argument is supplied, we update the permissions on the ViewMenu associated with this DAG according to the following rules: - If the role does not exist, we raise an exception. - If the role exists, we ensure that it has the specified set of permissions on the DAG - If any other permissions exist for the DAG that are not specified in `access_control`, we revoke them * Move RBAC constants to break circular dependency * Add license header * Sync DAG permissions via CLI and /refresh* endpoints Move the DAG-level permission syncing logic into `AirflowSecurityManager.sync_perm_for_dag`, and trigger this method from the CLI's `sync_perm` command and from the `/refresh` and `/refresh_all` web endpoints. * Default access_control to None (cherry picked from commit 5275a8f)
* [AIRFLOW-2694] Declare permissions in DAG definition This PR adds support for declaratively assigning DAG-level permissions to a role via the `DAG.__init__()` method. When the DAG definition is evaluated and the `access_control` argument is supplied, we update the permissions on the ViewMenu associated with this DAG according to the following rules: - If the role does not exist, we raise an exception. - If the role exists, we ensure that it has the specified set of permissions on the DAG - If any other permissions exist for the DAG that are not specified in `access_control`, we revoke them * Move RBAC constants to break circular dependency * Add license header * Sync DAG permissions via CLI and /refresh* endpoints Move the DAG-level permission syncing logic into `AirflowSecurityManager.sync_perm_for_dag`, and trigger this method from the CLI's `sync_perm` command and from the `/refresh` and `/refresh_all` web endpoints. * Default access_control to None (cherry picked from commit 5275a8f)
Make sure you have checked all steps below.
Jira
Description
This PR adds support for declaratively assigning DAG-level permissions
to a role via the
DAG.__init__()
method.When the DAG definition is evaluated and the
access_control
argumentis supplied, we update the permissions on the ViewMenu associated with
this DAG according to the following rules:
permissions on the DAG
access_control
, we revoke themTests
Added unit tests in
tests/models.py
to test that:Commits
Documentation
Not sure if we have any docs that need updating besides the docstrings on the DAG class. If so, let me know and I'll be happy to update those, as well.
Code Quality
flake8