From dd25379c4006ab26200729a71fecc5a82c582e05 Mon Sep 17 00:00:00 2001 From: Tao feng Date: Mon, 21 Jan 2019 11:19:12 -0800 Subject: [PATCH] [AIRFLOW-3745] Fix viewer not able to view dag details --- airflow/www/security.py | 20 ++++++++++++++------ docs/howto/add-new-role.rst | 1 + tests/www/test_views.py | 20 ++++++++++++++++++++ 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index f9f90f17944a7d..5ee2e8938d6ce1 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -140,7 +140,7 @@ ROLE_CONFIGS = [ { 'role': 'Viewer', - 'perms': viewer_perms, + 'perms': viewer_perms | dag_perms, 'vms': viewer_vms | dag_vms }, { @@ -355,11 +355,6 @@ def merge_pv(perm, view_menu): if pv.permission and pv.view_menu: all_pvs.add((pv.permission.name, pv.view_menu.name)) - # create perm for global logical dag - for dag in dag_vms: - for perm in dag_perms: - merge_pv(perm, dag) - # Get all the active / paused dags and insert them into a set all_dags_models = session.query(models.DagModel)\ .filter(or_(models.DagModel.is_active, models.DagModel.is_paused))\ @@ -424,6 +419,7 @@ def update_admin_perm_view(self): if p not in existing_perms_vms: existing_perms_vms.add(p) admin.permissions = list(existing_perms_vms) + self.get_session.commit() def sync_roles(self): @@ -435,6 +431,8 @@ def sync_roles(self): :return: None. """ logging.info('Start syncing user roles.') + # Create global all-dag VM + self.create_perm_vm_for_all_dag() # Create default user role. for config in ROLE_CONFIGS: @@ -460,3 +458,13 @@ def sync_perm_for_dag(self, dag_id): perm_on_dag = self.find_permission_view_menu(dag_perm, dag_id) if perm_on_dag is None: self.add_permission_view_menu(dag_perm, dag_id) + + def create_perm_vm_for_all_dag(self): + """ + Create perm-vm if not exist and insert into FAB security model for all-dags. + """ + # create perm for global logical dag + for dag_vm in dag_vms: + for perm in dag_perms: + self._merge_perm(permission_name=perm, + view_menu_name=dag_vm) diff --git a/docs/howto/add-new-role.rst b/docs/howto/add-new-role.rst index 8d221031859be9..60d718ed1eb770 100644 --- a/docs/howto/add-new-role.rst +++ b/docs/howto/add-new-role.rst @@ -31,3 +31,4 @@ and click ``List Roles`` in the new UI. The image shows a role which could only write to example_python_operator is created. And we could assign the given role to a new user using ``airflow users --add-role`` cli command. +Default roles(Admin, User, Viewer, Op) shiped with RBAC could view the details for every dag. diff --git a/tests/www/test_views.py b/tests/www/test_views.py index 0b7124f6710432..ac5a3e42f3f143 100644 --- a/tests/www/test_views.py +++ b/tests/www/test_views.py @@ -33,6 +33,7 @@ from flask._compat import PY2 from parameterized import parameterized from urllib.parse import quote_plus + from werkzeug.test import Client from airflow import configuration as conf @@ -1035,6 +1036,17 @@ def login(self, username=None, password=None): role=role_user, password='test_user') + role_viewer = self.appbuilder.sm.find_role('User') + test_viewer = self.appbuilder.sm.find_user(username='test_viewer') + if not test_viewer: + self.appbuilder.sm.add_user( + username='test_viewer', + first_name='test_viewer', + last_name='test_viewer', + email='test_viewer@fab.org', + role=role_viewer, + password='test_viewer') + dag_acl_role = self.appbuilder.sm.add_role('dag_acl_tester') dag_tester = self.appbuilder.sm.find_user(username='dag_tester') if not dag_tester: @@ -1522,6 +1534,14 @@ def test_log_success_for_user(self): self.check_content_in_response('"message":', resp) self.check_content_in_response('"metadata":', resp) + def test_tree_view_for_viewer(self): + self.logout() + self.login(username='test_viewer', + password='test_viewer') + url = 'tree?dag_id=example_bash_operator' + resp = self.client.get(url, follow_redirects=True) + self.check_content_in_response('runme_1', resp) + class TestTaskInstanceView(TestBase): TI_ENDPOINT = '/taskinstance/list/?_flt_0_execution_date={}'