From ba54a2fb910c913476c579532d1f8be0610e3903 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Wed, 15 Mar 2023 15:10:06 +0700 Subject: [PATCH 01/10] fix apache#25149 DAG.access_control can't sync when clean access_control --- airflow/models/dagbag.py | 6 +++--- airflow/www/security.py | 3 +-- tests/models/test_dagbag.py | 8 ++------ 3 files changed, 6 insertions(+), 11 deletions(-) diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index 24d61040f65c6..cfe8fdc80ae76 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -688,10 +688,10 @@ def _sync_perm_for_dag(cls, dag: DAG, session: Session = NEW_SESSION): root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id - def needs_perms(dag_id: str) -> bool: + def has_perms(dag_id: str) -> bool: dag_resource_name = resource_name_for_dag(dag_id) for permission_name in DAG_ACTIONS: - if not ( + if ( session.query(Permission) .join(Action) .join(Resource) @@ -702,7 +702,7 @@ def needs_perms(dag_id: str) -> bool: return True return False - if dag.access_control or needs_perms(root_dag_id): + if dag.access_control or has_perms(root_dag_id): cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) from airflow.www.security import ApplessAirflowSecurityManager diff --git a/airflow/www/security.py b/airflow/www/security.py index 6b2561292a6d9..d0d35ba055d9e 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -636,8 +636,7 @@ def sync_perm_for_dag(self, dag_id, access_control=None): for dag_action_name in self.DAG_ACTIONS: self.create_permission(dag_action_name, dag_resource_name) - if access_control: - self._sync_dag_view_permissions(dag_resource_name, access_control) + self._sync_dag_view_permissions(dag_resource_name, access_control) def _sync_dag_view_permissions(self, dag_id, access_control): """ diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index 3deb99ade5509..b9612a538fd91 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -908,7 +908,7 @@ def _sync_to_db(): def test_sync_perm_for_dag(self, mock_security_manager): """ Test that dagbag._sync_perm_for_dag will call ApplessAirflowSecurityManager.sync_perm_for_dag - when DAG specific perm views don't exist already or the DAG has access_control set. + when DAG specific perm views exist already or the DAG has access_control set. """ db_clean_up() with create_session() as session: @@ -926,11 +926,7 @@ def _sync_perms(): mock_sync_perm_for_dag.reset_mock() DagBag._sync_perm_for_dag(dag, session=session) - # perms dont exist - _sync_perms() - mock_sync_perm_for_dag.assert_called_once_with("test_example_bash_operator", None) - - # perms now exist + # perms dont exist and have not access_control _sync_perms() mock_sync_perm_for_dag.assert_not_called() From 3192ac581c57d2dbfaf0674e238ca0286169d675 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Sat, 18 Mar 2023 01:24:45 +0700 Subject: [PATCH 02/10] Use need perms to avoid circuit out on the first miss --- airflow/models/dagbag.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index cfe8fdc80ae76..6203d2fabfd2f 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -688,10 +688,10 @@ def _sync_perm_for_dag(cls, dag: DAG, session: Session = NEW_SESSION): root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id - def has_perms(dag_id: str) -> bool: + def needs_perms(dag_id: str) -> bool: dag_resource_name = resource_name_for_dag(dag_id) for permission_name in DAG_ACTIONS: - if ( + if not ( session.query(Permission) .join(Action) .join(Resource) @@ -702,7 +702,7 @@ def has_perms(dag_id: str) -> bool: return True return False - if dag.access_control or has_perms(root_dag_id): + if dag.access_control or not needs_perms(root_dag_id): cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) from airflow.www.security import ApplessAirflowSecurityManager From 4939b77e4954fb0703de1731e20f821216014801 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Mon, 20 Mar 2023 21:56:41 +0700 Subject: [PATCH 03/10] Sync perms in both cases --- airflow/models/dagbag.py | 29 +++++------------------------ tests/models/test_dagbag.py | 34 ---------------------------------- 2 files changed, 5 insertions(+), 58 deletions(-) diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index 6203d2fabfd2f..5003b0fc0c4d4 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -682,29 +682,10 @@ def sync_to_db(self, processor_subdir: str | None = None, session: Session = NEW @classmethod @provide_session def _sync_perm_for_dag(cls, dag: DAG, session: Session = NEW_SESSION): - """Sync DAG specific permissions, if necessary""" - from airflow.security.permissions import DAG_ACTIONS, resource_name_for_dag - from airflow.www.fab_security.sqla.models import Action, Permission, Resource - root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id - def needs_perms(dag_id: str) -> bool: - dag_resource_name = resource_name_for_dag(dag_id) - for permission_name in DAG_ACTIONS: - if not ( - session.query(Permission) - .join(Action) - .join(Resource) - .filter(Action.name == permission_name) - .filter(Resource.name == dag_resource_name) - .one_or_none() - ): - return True - return False - - if dag.access_control or not needs_perms(root_dag_id): - cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) - from airflow.www.security import ApplessAirflowSecurityManager - - security_manager = ApplessAirflowSecurityManager(session=session) - security_manager.sync_perm_for_dag(root_dag_id, dag.access_control) + cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) + from airflow.www.security import ApplessAirflowSecurityManager + + security_manager = ApplessAirflowSecurityManager(session=session) + security_manager.sync_perm_for_dag(root_dag_id, dag.access_control) diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index b9612a538fd91..3bdf4315ff789 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -44,7 +44,6 @@ from airflow.serialization.serialized_objects import SerializedDAG from airflow.utils.dates import timezone as tz from airflow.utils.session import create_session -from airflow.www.security import ApplessAirflowSecurityManager from tests import cluster_policies from tests.models import TEST_DAGS_FOLDER from tests.test_utils import db @@ -904,39 +903,6 @@ def _sync_to_db(): _sync_to_db() mock_sync_perm_for_dag.assert_called_once_with(dag, session=session) - @patch("airflow.www.security.ApplessAirflowSecurityManager") - def test_sync_perm_for_dag(self, mock_security_manager): - """ - Test that dagbag._sync_perm_for_dag will call ApplessAirflowSecurityManager.sync_perm_for_dag - when DAG specific perm views exist already or the DAG has access_control set. - """ - db_clean_up() - with create_session() as session: - security_manager = ApplessAirflowSecurityManager(session) - mock_sync_perm_for_dag = mock_security_manager.return_value.sync_perm_for_dag - mock_sync_perm_for_dag.side_effect = security_manager.sync_perm_for_dag - - dagbag = DagBag( - dag_folder=os.path.join(TEST_DAGS_FOLDER, "test_example_bash_operator.py"), - include_examples=False, - ) - dag = dagbag.dags["test_example_bash_operator"] - - def _sync_perms(): - mock_sync_perm_for_dag.reset_mock() - DagBag._sync_perm_for_dag(dag, session=session) - - # perms dont exist and have not access_control - _sync_perms() - mock_sync_perm_for_dag.assert_not_called() - - # Always sync if we have access_control - dag.access_control = {"Public": {"can_read"}} - _sync_perms() - mock_sync_perm_for_dag.assert_called_once_with( - "test_example_bash_operator", {"Public": {"can_read"}} - ) - @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_UPDATE_INTERVAL", 5) @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_FETCH_INTERVAL", 5) def test_get_dag_with_dag_serialization(self): From 34e2c6fb79bc1d20cf791fe2a5d439ab8b46eac1 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Tue, 21 Mar 2023 08:45:12 +0700 Subject: [PATCH 04/10] Check `access_control` before call `_get_or_create_dag_permission` --- airflow/www/security.py | 43 +++++++++++++++++++++-------------------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index d0d35ba055d9e..65279bfefc183 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -675,27 +675,28 @@ def _revoke_stale_permissions(resource: Resource): if resource: _revoke_stale_permissions(resource) - for rolename, action_names in access_control.items(): - role = self.find_role(rolename) - if not role: - raise AirflowException( - f"The access_control mapping for DAG '{dag_id}' includes a role named " - f"'{rolename}', but that role does not exist" - ) - - action_names = set(action_names) - invalid_action_names = action_names - self.DAG_ACTIONS - if invalid_action_names: - raise AirflowException( - f"The access_control map for DAG '{dag_resource_name}' includes " - f"the following invalid permissions: {invalid_action_names}; " - f"The set of valid permissions is: {self.DAG_ACTIONS}" - ) - - for action_name in action_names: - dag_perm = _get_or_create_dag_permission(action_name) - if dag_perm: - self.add_permission_to_role(role, dag_perm) + if access_control: + for rolename, action_names in access_control.items(): + role = self.find_role(rolename) + if not role: + raise AirflowException( + f"The access_control mapping for DAG '{dag_id}' includes a role named " + f"'{rolename}', but that role does not exist" + ) + + action_names = set(action_names) + invalid_action_names = action_names - self.DAG_ACTIONS + if invalid_action_names: + raise AirflowException( + f"The access_control map for DAG '{dag_resource_name}' includes " + f"the following invalid permissions: {invalid_action_names}; " + f"The set of valid permissions is: {self.DAG_ACTIONS}" + ) + + for action_name in action_names: + dag_perm = _get_or_create_dag_permission(action_name) + if dag_perm: + self.add_permission_to_role(role, dag_perm) def create_perm_vm_for_all_dag(self): """Create perm-vm if not exist and insert into FAB security model for all-dags.""" From 6bc89227c53b5f78900319cb7895922052596cfb Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Wed, 22 Mar 2023 16:20:08 +0700 Subject: [PATCH 05/10] Call `sync_perm_for_dag` in both cases --- airflow/www/security.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index 65279bfefc183..3bf086d4c8fc2 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -567,8 +567,7 @@ def create_dag_specific_permissions(self) -> None: if (action_name, dag_resource_name) not in perms: self._merge_perm(action_name, dag_resource_name) - if dag.access_control: - self.sync_perm_for_dag(dag_resource_name, dag.access_control) + self.sync_perm_for_dag(dag_resource_name, dag.access_control) def update_admin_permission(self): """ From 32c6a1930377fefdd78728c1d9e4c6fc43a9f97a Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Wed, 22 Mar 2023 20:07:53 +0700 Subject: [PATCH 06/10] Update security.py --- airflow/www/security.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index 3bf086d4c8fc2..f3d613065c1d7 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -567,7 +567,8 @@ def create_dag_specific_permissions(self) -> None: if (action_name, dag_resource_name) not in perms: self._merge_perm(action_name, dag_resource_name) - self.sync_perm_for_dag(dag_resource_name, dag.access_control) + if dag.access_control: + self.sync_perm_for_dag(dag_resource_name, dag.access_control) def update_admin_permission(self): """ @@ -635,7 +636,8 @@ def sync_perm_for_dag(self, dag_id, access_control=None): for dag_action_name in self.DAG_ACTIONS: self.create_permission(dag_action_name, dag_resource_name) - self._sync_dag_view_permissions(dag_resource_name, access_control) + if access_control: + self._sync_dag_view_permissions(dag_resource_name, access_control) def _sync_dag_view_permissions(self, dag_id, access_control): """ @@ -674,7 +676,6 @@ def _revoke_stale_permissions(resource: Resource): if resource: _revoke_stale_permissions(resource) - if access_control: for rolename, action_names in access_control.items(): role = self.find_role(rolename) if not role: From 587158832c41e6781d04202219f57f265098bead Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Wed, 22 Mar 2023 20:09:22 +0700 Subject: [PATCH 07/10] Update security.py --- airflow/www/security.py | 42 ++++++++++++++++++++--------------------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index f3d613065c1d7..6b2561292a6d9 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -676,27 +676,27 @@ def _revoke_stale_permissions(resource: Resource): if resource: _revoke_stale_permissions(resource) - for rolename, action_names in access_control.items(): - role = self.find_role(rolename) - if not role: - raise AirflowException( - f"The access_control mapping for DAG '{dag_id}' includes a role named " - f"'{rolename}', but that role does not exist" - ) - - action_names = set(action_names) - invalid_action_names = action_names - self.DAG_ACTIONS - if invalid_action_names: - raise AirflowException( - f"The access_control map for DAG '{dag_resource_name}' includes " - f"the following invalid permissions: {invalid_action_names}; " - f"The set of valid permissions is: {self.DAG_ACTIONS}" - ) - - for action_name in action_names: - dag_perm = _get_or_create_dag_permission(action_name) - if dag_perm: - self.add_permission_to_role(role, dag_perm) + for rolename, action_names in access_control.items(): + role = self.find_role(rolename) + if not role: + raise AirflowException( + f"The access_control mapping for DAG '{dag_id}' includes a role named " + f"'{rolename}', but that role does not exist" + ) + + action_names = set(action_names) + invalid_action_names = action_names - self.DAG_ACTIONS + if invalid_action_names: + raise AirflowException( + f"The access_control map for DAG '{dag_resource_name}' includes " + f"the following invalid permissions: {invalid_action_names}; " + f"The set of valid permissions is: {self.DAG_ACTIONS}" + ) + + for action_name in action_names: + dag_perm = _get_or_create_dag_permission(action_name) + if dag_perm: + self.add_permission_to_role(role, dag_perm) def create_perm_vm_for_all_dag(self): """Create perm-vm if not exist and insert into FAB security model for all-dags.""" From a15fa4ef73fa8e9151bc72619fa83a1bf2a36248 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Thu, 23 Mar 2023 09:13:42 +0700 Subject: [PATCH 08/10] Update test_dagbag.py --- tests/models/test_dagbag.py | 38 +++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index 6aa86ab008942..09d52c3275400 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -44,6 +44,7 @@ from airflow.serialization.serialized_objects import SerializedDAG from airflow.utils.dates import timezone as tz from airflow.utils.session import create_session +from airflow.www.security import ApplessAirflowSecurityManager from tests import cluster_policies from tests.models import TEST_DAGS_FOLDER from tests.test_utils import db @@ -903,6 +904,43 @@ def _sync_to_db(): _sync_to_db() mock_sync_perm_for_dag.assert_called_once_with(dag, session=session) + @patch("airflow.www.security.ApplessAirflowSecurityManager") + def test_sync_perm_for_dag(self, mock_security_manager): + """ + Test that dagbag._sync_perm_for_dag will call ApplessAirflowSecurityManager.sync_perm_for_dag + when DAG specific perm views don't exist already or the DAG has access_control set. + """ + db_clean_up() + with create_session() as session: + security_manager = ApplessAirflowSecurityManager(session) + mock_sync_perm_for_dag = mock_security_manager.return_value.sync_perm_for_dag + mock_sync_perm_for_dag.side_effect = security_manager.sync_perm_for_dag + + dagbag = DagBag( + dag_folder=os.path.join(TEST_DAGS_FOLDER, "test_example_bash_operator.py"), + include_examples=False, + ) + dag = dagbag.dags["test_example_bash_operator"] + + def _sync_perms(): + mock_sync_perm_for_dag.reset_mock() + DagBag._sync_perm_for_dag(dag, session=session) + + # perms dont exist + _sync_perms() + mock_sync_perm_for_dag.assert_called_once_with("test_example_bash_operator", None) + + # perms now exist + _sync_perms() + mock_sync_perm_for_dag.assert_called_once_with("test_example_bash_operator", None) + + # Always sync if we have access_control + dag.access_control = {"Public": {"can_read"}} + _sync_perms() + mock_sync_perm_for_dag.assert_called_once_with( + "test_example_bash_operator", {"Public": {"can_read"}} + ) + @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_UPDATE_INTERVAL", 5) @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_FETCH_INTERVAL", 5) def test_get_dag_with_dag_serialization(self): From 66e7cb91938c52deb883e0ca4dd2f8d011e867a8 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Thu, 23 Mar 2023 11:42:34 +0700 Subject: [PATCH 09/10] reset permission if `access_control` is empty --- airflow/www/security.py | 59 ++++++++++++++++++++++++----------------- 1 file changed, 34 insertions(+), 25 deletions(-) diff --git a/airflow/www/security.py b/airflow/www/security.py index 6b2561292a6d9..31df228ba7eab 100644 --- a/airflow/www/security.py +++ b/airflow/www/security.py @@ -636,8 +636,7 @@ def sync_perm_for_dag(self, dag_id, access_control=None): for dag_action_name in self.DAG_ACTIONS: self.create_permission(dag_action_name, dag_resource_name) - if access_control: - self._sync_dag_view_permissions(dag_resource_name, access_control) + self._sync_dag_view_permissions(dag_resource_name, access_control) def _sync_dag_view_permissions(self, dag_id, access_control): """ @@ -662,8 +661,17 @@ def _revoke_stale_permissions(resource: Resource): for perm in existing_dag_perms: non_admin_roles = [role for role in perm.role if role.name != "Admin"] for role in non_admin_roles: - target_perms_for_role = access_control.get(role.name, {}) - if perm.action.name not in target_perms_for_role: + if access_control: + target_perms_for_role = access_control.get(role.name, {}) + if perm.action.name not in target_perms_for_role: + self.log.info( + "Revoking '%s' on DAG '%s' for role '%s'", + perm.action, + dag_resource_name, + role.name, + ) + self.remove_permission_from_role(role, perm) + else: self.log.info( "Revoking '%s' on DAG '%s' for role '%s'", perm.action, @@ -676,27 +684,28 @@ def _revoke_stale_permissions(resource: Resource): if resource: _revoke_stale_permissions(resource) - for rolename, action_names in access_control.items(): - role = self.find_role(rolename) - if not role: - raise AirflowException( - f"The access_control mapping for DAG '{dag_id}' includes a role named " - f"'{rolename}', but that role does not exist" - ) - - action_names = set(action_names) - invalid_action_names = action_names - self.DAG_ACTIONS - if invalid_action_names: - raise AirflowException( - f"The access_control map for DAG '{dag_resource_name}' includes " - f"the following invalid permissions: {invalid_action_names}; " - f"The set of valid permissions is: {self.DAG_ACTIONS}" - ) - - for action_name in action_names: - dag_perm = _get_or_create_dag_permission(action_name) - if dag_perm: - self.add_permission_to_role(role, dag_perm) + if access_control: + for rolename, action_names in access_control.items(): + role = self.find_role(rolename) + if not role: + raise AirflowException( + f"The access_control mapping for DAG '{dag_id}' includes a role named " + f"'{rolename}', but that role does not exist" + ) + + action_names = set(action_names) + invalid_action_names = action_names - self.DAG_ACTIONS + if invalid_action_names: + raise AirflowException( + f"The access_control map for DAG '{dag_resource_name}' includes " + f"the following invalid permissions: {invalid_action_names}; " + f"The set of valid permissions is: {self.DAG_ACTIONS}" + ) + + for action_name in action_names: + dag_perm = _get_or_create_dag_permission(action_name) + if dag_perm: + self.add_permission_to_role(role, dag_perm) def create_perm_vm_for_all_dag(self): """Create perm-vm if not exist and insert into FAB security model for all-dags.""" From 468c67d63bcef73ed154367556e45040425d5912 Mon Sep 17 00:00:00 2001 From: Huy Mac Date: Mon, 27 Mar 2023 18:10:23 +0700 Subject: [PATCH 10/10] Fix test_view_home.py --- tests/www/views/test_views_home.py | 48 ++++++++++++++++++++++++++++-- 1 file changed, 45 insertions(+), 3 deletions(-) diff --git a/tests/www/views/test_views_home.py b/tests/www/views/test_views_home.py index 6619bcdf71515..018f2a6406e91 100644 --- a/tests/www/views/test_views_home.py +++ b/tests/www/views/test_views_home.py @@ -155,6 +155,38 @@ def working_dags(tmpdir): _process_file(filename, session) +@pytest.fixture() +def working_dags_with_edit_perm(tmpdir): + dag_contents_template = "from airflow import DAG\ndag = DAG('{}', tags=['{}'])" + + with create_session() as session: + for dag_id, tag in list(zip(TEST_FILTER_DAG_IDS, TEST_TAGS)): + filename = os.path.join(tmpdir, f"{dag_id}.py") + if dag_id == 'filter_test_1': + with open(filename, "w") as f: + f.writelines("from airflow import DAG\ndag = DAG('{}', tags=['{}'],".format(dag_id, tag) + "access_control={'role_single_dag':{'can_edit'}})" ) + else: + with open(filename, "w") as f: + f.writelines(dag_contents_template.format(dag_id, tag)) + _process_file(filename, session) + + +@pytest.fixture() +def working_dags_with_read_perm(tmpdir): + dag_contents_template = "from airflow import DAG\ndag = DAG('{}', tags=['{}'])" + + with create_session() as session: + for dag_id, tag in list(zip(TEST_FILTER_DAG_IDS, TEST_TAGS)): + filename = os.path.join(tmpdir, f"{dag_id}.py") + if dag_id == 'filter_test_1': + with open(filename, "w") as f: + f.writelines("from airflow import DAG\ndag = DAG('{}', tags=['{}'],".format(dag_id, tag) + "access_control={'role_single_dag':{'can_read'}})" ) + else: + with open(filename, "w") as f: + f.writelines(dag_contents_template.format(dag_id, tag)) + _process_file(filename, session) + + @pytest.fixture() def broken_dags(tmpdir, working_dags): with create_session() as session: @@ -165,6 +197,16 @@ def broken_dags(tmpdir, working_dags): _process_file(filename, session) +@pytest.fixture() +def broken_dags_with_read_perm(tmpdir, working_dags_with_read_perm): + with create_session() as session: + for dag_id in TEST_FILTER_DAG_IDS: + filename = os.path.join(tmpdir, f"{dag_id}.py") + with open(filename, "w") as f: + f.writelines("airflow DAG") + _process_file(filename, session) + + def test_home_filter_tags(working_dags, admin_client): with admin_client: admin_client.get("home?tags=example&tags=data", follow_redirects=True) @@ -183,7 +225,7 @@ def test_home_importerrors(broken_dags, user_client): @pytest.mark.parametrize("page", ["home", "home?status=active", "home?status=paused", "home?status=all"]) -def test_home_importerrors_filtered_singledag_user(broken_dags, client_single_dag, page): +def test_home_importerrors_filtered_singledag_user(broken_dags_with_read_perm, client_single_dag, page): # Users that can only see certain DAGs get a filtered list of import errors resp = client_single_dag.get(page, follow_redirects=True) check_content_in_response("Import Errors", resp) @@ -201,7 +243,7 @@ def test_home_dag_list(working_dags, user_client): check_content_in_response(f"dag_id={dag_id}", resp) -def test_home_dag_list_filtered_singledag_user(working_dags, client_single_dag): +def test_home_dag_list_filtered_singledag_user(working_dags_with_read_perm, client_single_dag): # Users that can only see certain DAGs get a filtered list resp = client_single_dag.get("home", follow_redirects=True) # They can see the first DAG @@ -219,7 +261,7 @@ def test_home_dag_list_search(working_dags, user_client): check_content_not_in_response("dag_id=a_first_dag_id_asc", resp) -def test_home_dag_edit_permissions(capture_templates, working_dags, client_single_dag_edit): +def test_home_dag_edit_permissions(capture_templates, working_dags_with_edit_perm, client_single_dag_edit): with capture_templates() as templates: client_single_dag_edit.get("home", follow_redirects=True)