Skip to content
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

Merged
merged 5 commits into from
Feb 6, 2019

Conversation

astahlman
Copy link
Contributor

Make sure you have checked all steps below.

Jira

  • [ X ] My PR addresses the following Airflow Jira issues and references them in the PR title. For example, "[AIRFLOW-XXX] My Airflow PR"

Description

  • [ X ] Here are some details about my PR, including screenshots of any UI changes:

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

Tests

  • [ X ] My PR adds the following unit tests OR does not need testing for this extremely good reason:

Added unit tests in tests/models.py to test that:

  • Invalid roles raise an exception
  • Invalid permissions raise an exception
  • New permissions in the access_control map are created
  • Existing permissions that are not specified in the access_control map are revoked

Commits

  • [ X ] My commits all reference Jira issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "How to write a good git commit message":
    1. Subject is separated from body by a blank line
    2. Subject is limited to 50 characters (not including Jira issue reference)
    3. Subject does not end with a period
    4. Subject uses the imperative mood ("add", not "adding")
    5. Body wraps at 72 characters
    6. Body explains "what" and "why", not "how"

Documentation

  • [ X ] In case of new functionality, my PR adds documentation that describes how to use it.
    • When adding new operators/hooks/sensors, the autoclass documentation generation needs to be added.
    • All the public functions and the classes in the PR contain docstrings that explain what it does

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

  • [ X ] Passes flake8

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
@feng-tao feng-tao changed the title [AIRFLOW-2694] Declare permissions in DAG definition [WIP] [AIRFLOW-2694] Declare permissions in DAG definition Feb 2, 2019
@feng-tao
Copy link
Member

feng-tao commented Feb 2, 2019

hey @astahlman , will take a look next week.

@codecov-io
Copy link

codecov-io commented Feb 2, 2019

Codecov Report

Merging #4642 into master will decrease coverage by 0.03%.
The diff coverage is 94.11%.

Impacted file tree graph

@@            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
Impacted Files Coverage Δ
airflow/www/views.py 76.12% <100%> (+0.01%) ⬆️
airflow/bin/cli.py 66.73% <100%> (+0.14%) ⬆️
airflow/models/__init__.py 93% <90%> (-0.13%) ⬇️
airflow/www/security.py 93.12% <94.11%> (+0.17%) ⬆️
airflow/contrib/operators/sftp_to_s3_operator.py 77.14% <0%> (-22.86%) ⬇️
airflow/contrib/operators/s3_to_sftp_operator.py 77.77% <0%> (-22.23%) ⬇️
airflow/contrib/operators/databricks_operator.py 82.67% <0%> (-7.85%) ⬇️
airflow/models/log.py 100% <0%> (ø)

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update f07f3a8...b85796d. Read the comment docs.

:type access_control: dict

"""
sm = cached_appbuilder().sm
Copy link
Member

@feng-tao feng-tao Feb 4, 2019

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:

  1. 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.
  2. 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.

cc @jgao54 @mistercrunch

Copy link
Contributor Author

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()?

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()

def sync_perm(args):

Copy link
Member

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.

Copy link
Member

@feng-tao feng-tao Feb 5, 2019

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.

Copy link
Contributor Author

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.
Copy link
Member

@feng-tao feng-tao left a 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.

@feng-tao feng-tao changed the title [WIP] [AIRFLOW-2694] Declare permissions in DAG definition [AIRFLOW-2694] Declare permissions in DAG definition Feb 5, 2019
airflow/www/security.py Outdated Show resolved Hide resolved
@feng-tao
Copy link
Member

feng-tao commented Feb 5, 2019

Once this pr is merged, I think we could declare Airflow DAG level support is done :)

@feng-tao
Copy link
Member

feng-tao commented Feb 6, 2019

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 .

@feng-tao feng-tao merged commit 5275a8f into apache:master Feb 6, 2019
@ashb
Copy link
Member

ashb commented Mar 5, 2019

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:

  • We add a column/columns to the Dag table (can_edit_roles, can_read_roles) and the scheduler updates that
  • Update security manger to look in there when checking the can_dag_* permissions instead of AB's own role<->perm mapping?

Dunno. This was unexpected is all.

@ashb
Copy link
Member

ashb commented Mar 5, 2019

Also (not sure it relates to this PR or another one) but what happens if I specify a dag_id="all_dags" and run sync_perms :D

@feng-tao
Copy link
Member

feng-tao commented Mar 5, 2019

@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:

  1. we create new roles;
  2. The dag file declare permissions for certain role which those roles will have that permissions.
    // others details could be found in the pr descriptions.

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.

@astahlman
Copy link
Contributor Author

@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 access_control settings are updated. Of course, we could add some automation to our deployment process to run the sync permissions command whenever DAG definitions change, but I like the idea of the scheduler handling this automatically.

What do you think @feng-tao?

@feng-tao
Copy link
Member

feng-tao commented Mar 6, 2019

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:

  1. moved all the FAB models inside Airflow model, which scheduler is aware of the FAB models. And scheduler could directly update the tables in this case. Not a very ideal approach as it will fail once FAB models get updated which requires Airflow updated the model file as well. But ideally it should work with this approach.
  2. RBAC is strictly a web construct(Max B's term) which means scheduler is unaware of roles & users, maybe as the supervisor it periodically calls a REST endpoint to trigger creating new perms from new DAGs. This approach has latency / performance concern.

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.

wmorris75 pushed a commit to modmed/incubator-airflow that referenced this pull request Jul 29, 2019
* [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
@icyfox-bupt
Copy link
Contributor

Hi, @feng-tao and @astahlman :

Thanks for this commit and I wonder why this function didn't implement on rbac views ?
I'm working on www_rbac and I didn't find the same changes in www_rbac folder in this commit, should we add the same logical for rbac view?

I'm using RBAC view and have 200+ DAGs, which made my group permission explosion..
(Actually, When each DAG be added into airflow, a 'can_dag_read' and a 'can_dag_edit' permission will add to DAG view, which made my Admin role have 500+ permission_views...)

@ashb
Copy link
Member

ashb commented Sep 25, 2019

@icyfox-bupt Airflow's "release workflow" is that all commits are made to master and then cherry-picked back on to the release branch (v1-10-stable in this case).

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 :)

@icyfox-bupt
Copy link
Contributor

@icyfox-bupt Airflow's "release workflow" is that all commits are made to master and then cherry-picked back on to the release branch (v1-10-stable in this case).

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~!

@feng-tao
Copy link
Member

@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.

@ashb
Copy link
Member

ashb commented Sep 25, 2019

@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.

kaxil pushed a commit that referenced this pull request Dec 17, 2019
* [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)
kaxil pushed a commit that referenced this pull request Dec 18, 2019
* [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)
ashb pushed a commit that referenced this pull request Dec 19, 2019
* [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)
@eschachar eschachar deleted the astahlman/airflow-2694 branch September 24, 2022 22:33
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants