You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ep...@apache.org on 2021/04/05 09:22:56 UTC

[airflow] branch master updated: Add CUD REST API endpoints for Roles (#14840)

This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new 6706b67  Add CUD REST API endpoints for Roles (#14840)
6706b67 is described below

commit 6706b67fecc00a22c1e1d6658616ed9dd96bbc7b
Author: Ephraim Anierobi <sp...@gmail.com>
AuthorDate: Mon Apr 5 10:22:43 2021 +0100

    Add CUD REST API endpoints for Roles (#14840)
---
 .../endpoints/role_and_permission_endpoint.py      |  89 +++++-
 airflow/api_connexion/openapi/v1.yaml              |  72 +++++
 airflow/security/permissions.py                    |   1 +
 .../endpoints/test_role_and_permission_endpoint.py | 330 ++++++++++++++++++++-
 4 files changed, 489 insertions(+), 3 deletions(-)

diff --git a/airflow/api_connexion/endpoints/role_and_permission_endpoint.py b/airflow/api_connexion/endpoints/role_and_permission_endpoint.py
index 9668e4d..b8e0691 100644
--- a/airflow/api_connexion/endpoints/role_and_permission_endpoint.py
+++ b/airflow/api_connexion/endpoints/role_and_permission_endpoint.py
@@ -14,12 +14,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from flask import current_app
+
+from connexion import NoContent
+from flask import current_app, request
 from flask_appbuilder.security.sqla.models import Permission, Role
+from marshmallow import ValidationError
 from sqlalchemy import func
 
 from airflow.api_connexion import security
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, BadRequest, NotFound
 from airflow.api_connexion.parameters import apply_sorting, check_limit, format_parameters
 from airflow.api_connexion.schemas.role_and_permission_schema import (
     ActionCollection,
@@ -31,6 +34,19 @@ from airflow.api_connexion.schemas.role_and_permission_schema import (
 from airflow.security import permissions
 
 
+def _check_action_and_resource(sm, perms):
+    """
+    Checks if the action or resource exists and raise 400 if not
+
+    This function is intended for use in the REST API because it raise 400
+    """
+    for item in perms:
+        if not sm.find_permission(item[0]):
+            raise BadRequest(detail=f"The specified action: '{item[0]}' was not found")
+        if not sm.find_view_menu(item[1]):
+            raise BadRequest(detail=f"The specified resource: '{item[1]}' was not found")
+
+
 @security.requires_access([(permissions.ACTION_CAN_SHOW, permissions.RESOURCE_ROLE_MODEL_VIEW)])
 def get_role(role_name):
     """Get role"""
@@ -66,3 +82,72 @@ def get_permissions(limit=None, offset=None):
     query = session.query(Permission)
     actions = query.offset(offset).limit(limit).all()
     return action_collection_schema.dump(ActionCollection(actions=actions, total_entries=total_entries))
+
+
+@security.requires_access([(permissions.ACTION_CAN_DELETE, permissions.RESOURCE_ROLE_MODEL_VIEW)])
+def delete_role(role_name):
+    """Delete a role"""
+    ab_security_manager = current_app.appbuilder.sm
+    role = ab_security_manager.find_role(name=role_name)
+    if not role:
+        raise NotFound(title="Role not found", detail=f"The Role with name `{role_name}` was not found")
+    ab_security_manager.delete_role(role_name=role_name)
+    return NoContent, 204
+
+
+@security.requires_access([(permissions.ACTION_CAN_EDIT, permissions.RESOURCE_ROLE_MODEL_VIEW)])
+def patch_role(role_name, update_mask=None):
+    """Update a role"""
+    appbuilder = current_app.appbuilder
+    security_manager = appbuilder.sm
+    body = request.json
+    try:
+        data = role_schema.load(body)
+    except ValidationError as err:
+        raise BadRequest(detail=str(err.messages))
+    role = security_manager.find_role(name=role_name)
+    if not role:
+        raise NotFound(title="Role not found", detail=f"Role with name: `{role_name} was not found")
+    if update_mask:
+        update_mask = [i.strip() for i in update_mask]
+        data_ = {}
+        for field in update_mask:
+            if field in data and not field == "permissions":
+                data_[field] = data[field]
+            elif field == "actions":
+                data_["permissions"] = data['permissions']
+            else:
+                raise BadRequest(detail=f"'{field}' in update_mask is unknown")
+        data = data_
+    perms = data.get("permissions", [])
+    if perms:
+        perms = [
+            (item['permission']['name'], item['view_menu']['name']) for item in data['permissions'] if item
+        ]
+        _check_action_and_resource(security_manager, perms)
+    security_manager.update_role(pk=role.id, name=data['name'])
+    security_manager.init_role(role_name=data['name'], perms=perms or role.permissions)
+    return role_schema.dump(role)
+
+
+@security.requires_access([(permissions.ACTION_CAN_ADD, permissions.RESOURCE_ROLE_MODEL_VIEW)])
+def post_role():
+    """Create a new role"""
+    appbuilder = current_app.appbuilder
+    security_manager = appbuilder.sm
+    body = request.json
+    try:
+        data = role_schema.load(body)
+    except ValidationError as err:
+        raise BadRequest(detail=str(err.messages))
+    role = security_manager.find_role(name=data['name'])
+    if not role:
+        perms = [
+            (item['permission']['name'], item['view_menu']['name']) for item in data['permissions'] if item
+        ]
+        _check_action_and_resource(security_manager, perms)
+        security_manager.init_role(role_name=data['name'], perms=perms)
+        return role_schema.dump(role)
+    raise AlreadyExists(
+        detail=f"Role with name `{role.name}` already exist. Please update with patch endpoint"
+    )
diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml
index 0d90059..d8944bc 100644
--- a/airflow/api_connexion/openapi/v1.yaml
+++ b/airflow/api_connexion/openapi/v1.yaml
@@ -1412,6 +1412,31 @@ paths:
         '403':
           $ref: '#/components/responses/PermissionDenied'
 
+    post:
+      summary: Create a role
+      x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
+      operationId: post_role
+      tags: [Role]
+      requestBody:
+        required: true
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/Role'
+      responses:
+        '200':
+          description: Success.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/Role'
+        '400':
+          $ref: '#/components/responses/BadRequest'
+        '401':
+          $ref: '#/components/responses/Unauthenticated'
+        '403':
+          $ref: '#/components/responses/PermissionDenied'
+
   /roles/{role_name}:
     parameters:
       - $ref: '#/components/parameters/RoleName'
@@ -1435,6 +1460,53 @@ paths:
         '404':
           $ref: '#/components/responses/NotFound'
 
+    patch:
+      summary: Update a role
+      x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
+      operationId: patch_role
+      tags: [Role]
+      parameters:
+        - $ref: '#/components/parameters/UpdateMask'
+      requestBody:
+        required: true
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/Role'
+
+      responses:
+        '200':
+          description: Success.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/Role'
+        '400':
+          $ref: '#/components/responses/BadRequest'
+        '401':
+          $ref: '#/components/responses/Unauthenticated'
+        '403':
+          $ref: '#/components/responses/PermissionDenied'
+        '404':
+          $ref: '#/components/responses/NotFound'
+
+    delete:
+      summary: Delete a role
+      x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
+      operationId: delete_role
+      tags: [Role]
+      responses:
+        '204':
+          description: Success.
+        '400':
+          $ref: '#/components/responses/BadRequest'
+        '401':
+          $ref: '#/components/responses/Unauthenticated'
+        '403':
+          $ref: '#/components/responses/PermissionDenied'
+        '404':
+          $ref: '#/components/responses/NotFound'
+
   /permissions:
     get:
       summary: List permissions
diff --git a/airflow/security/permissions.py b/airflow/security/permissions.py
index 1a49e11..ffa4457 100644
--- a/airflow/security/permissions.py
+++ b/airflow/security/permissions.py
@@ -50,6 +50,7 @@ RESOURCE_ROLE_MODEL_VIEW = "RoleModelView"
 RESOURCE_PERMISSION_MODEL_VIEW = "PermissionModelView"
 
 # Action Constants
+ACTION_CAN_ADD = "can_add"
 ACTION_CAN_LIST = "can_list"
 ACTION_CAN_SHOW = "can_show"
 ACTION_CAN_CREATE = "can_create"
diff --git a/tests/api_connexion/endpoints/test_role_and_permission_endpoint.py b/tests/api_connexion/endpoints/test_role_and_permission_endpoint.py
index 4881bf4..6b07282 100644
--- a/tests/api_connexion/endpoints/test_role_and_permission_endpoint.py
+++ b/tests/api_connexion/endpoints/test_role_and_permission_endpoint.py
@@ -16,12 +16,19 @@
 # under the License.
 
 import pytest
+from flask_appbuilder.security.sqla.models import Role
 from parameterized import parameterized
 
 from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP
 from airflow.security import permissions
 from airflow.www.security import EXISTING_ROLES
-from tests.test_utils.api_connexion_utils import assert_401, create_user, delete_user
+from tests.test_utils.api_connexion_utils import (
+    assert_401,
+    create_role,
+    create_user,
+    delete_role,
+    delete_user,
+)
 
 
 @pytest.fixture(scope="module")
@@ -33,8 +40,11 @@ def configured_app(minimal_app_for_api):
         role_name="Test",
         permissions=[
             (permissions.ACTION_CAN_LIST, permissions.RESOURCE_ROLE_MODEL_VIEW),
+            (permissions.ACTION_CAN_ADD, permissions.RESOURCE_ROLE_MODEL_VIEW),
             (permissions.ACTION_CAN_SHOW, permissions.RESOURCE_ROLE_MODEL_VIEW),
             (permissions.ACTION_CAN_LIST, permissions.RESOURCE_PERMISSION_MODEL_VIEW),
+            (permissions.ACTION_CAN_DELETE, permissions.RESOURCE_ROLE_MODEL_VIEW),
+            (permissions.ACTION_CAN_EDIT, permissions.RESOURCE_ROLE_MODEL_VIEW),
         ],
     )
     create_user(app, username="test_no_permissions", role_name="TestNoPermissions")  # type: ignore
@@ -50,6 +60,18 @@ class TestRoleEndpoint:
         self.app = configured_app
         self.client = self.app.test_client()  # type:ignore
 
+    def teardown_method(self):
+        """
+        Delete all roles except these ones.
+        Test and TestNoPermissions are deleted by delete_user above
+        """
+        session = self.app.appbuilder.get_session
+        existing_roles = set(EXISTING_ROLES)
+        existing_roles.update(['Test', 'TestNoPermissions'])
+        roles = session.query(Role).filter(~Role.name.in_(existing_roles)).all()
+        for role in roles:
+            delete_role(self.app, role.name)
+
 
 class TestGetRoleEndpoint(TestRoleEndpoint):
     def test_should_response_200(self):
@@ -154,3 +176,309 @@ class TestGetPermissionsEndpoint(TestRoleEndpoint):
             "/api/v1/permissions", environ_overrides={'REMOTE_USER': "test_no_permissions"}
         )
         assert response.status_code == 403
+
+
+class TestPostRole(TestRoleEndpoint):
+    def test_post_should_respond_200(self):
+        payload = {
+            'name': 'Test2',
+            'actions': [{'resource': {'name': 'Connections'}, 'action': {'name': 'can_create'}}],
+        }
+        response = self.client.post("/api/v1/roles", json=payload, environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 200
+        role = self.app.appbuilder.sm.find_role('Test2')
+        assert role is not None
+
+    @parameterized.expand(
+        [
+            (
+                {
+                    'actions': [{'resource': {'name': 'Connections'}, 'action': {'name': 'can_create'}}],
+                },
+                "{'name': ['Missing data for required field.']}",
+            ),
+            (
+                {
+                    'name': "TestRole",
+                    'actionss': [
+                        {
+                            'resource': {'name': 'Connections'},  # actionss not correct
+                            'action': {'name': 'can_create'},
+                        }
+                    ],
+                },
+                "{'actionss': ['Unknown field.']}",
+            ),
+            (
+                {
+                    'name': "TestRole",
+                    'actions': [
+                        {
+                            'resources': {'name': 'Connections'},  # resources is invalid, should be resource
+                            'action': {'name': 'can_create'},
+                        }
+                    ],
+                },
+                "{'actions': {0: {'resources': ['Unknown field.']}}}",
+            ),
+            (
+                {
+                    'name': "TestRole",
+                    'actions': [
+                        {'resource': {'name': 'Connections'}, 'actions': {'name': 'can_create'}}
+                    ],  # actions is invalid, should be action
+                },
+                "{'actions': {0: {'actions': ['Unknown field.']}}}",
+            ),
+            (
+                {
+                    'name': "TestRole",
+                    'actions': [
+                        {
+                            'resource': {'name': 'FooBars'},  # FooBars is not a resource
+                            'action': {'name': 'can_create'},
+                        }
+                    ],
+                },
+                "The specified resource: 'FooBars' was not found",
+            ),
+            (
+                {
+                    'name': "TestRole",
+                    'actions': [
+                        {'resource': {'name': 'Connections'}, 'action': {'name': 'can_amend'}}
+                    ],  # can_amend is not an action
+                },
+                "The specified action: 'can_amend' was not found",
+            ),
+        ]
+    )
+    def test_post_should_respond_400_for_invalid_payload(self, payload, error_message):
+        response = self.client.post("/api/v1/roles", json=payload, environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 400
+        assert response.json == {
+            'detail': error_message,
+            'status': 400,
+            'title': 'Bad Request',
+            'type': EXCEPTIONS_LINK_MAP[400],
+        }
+
+    def test_post_should_respond_409_already_exist(self):
+        payload = {
+            'name': 'Test',
+            'actions': [{'resource': {'name': 'Connections'}, 'action': {'name': 'can_create'}}],
+        }
+        response = self.client.post("/api/v1/roles", json=payload, environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 409
+        assert response.json == {
+            'detail': "Role with name `Test` already exist. Please update with patch endpoint",
+            'status': 409,
+            'title': 'Conflict',
+            'type': EXCEPTIONS_LINK_MAP[409],
+        }
+
+    def test_should_raises_401_unauthenticated(self):
+        response = self.client.post(
+            "/api/v1/roles",
+            json={
+                'name': 'Test2',
+                'actions': [{'resource': {'name': 'Connections'}, 'action': {'name': 'can_create'}}],
+            },
+        )
+
+        assert_401(response)
+
+    def test_should_raise_403_forbidden(self):
+        response = self.client.post(
+            "/api/v1/roles",
+            json={
+                "name": "mytest2",
+                "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+            },
+            environ_overrides={'REMOTE_USER': "test_no_permissions"},
+        )
+        assert response.status_code == 403
+
+
+class TestDeleteRole(TestRoleEndpoint):
+    def test_delete_should_respond_204(self, session):
+        role = create_role(self.app, "mytestrole")
+        response = self.client.delete(f"/api/v1/roles/{role.name}", environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 204
+        role_obj = session.query(Role).filter(Role.name == role.name).all()
+        assert len(role_obj) == 0
+
+    def test_delete_should_respond_404(self):
+        response = self.client.delete(
+            "/api/v1/roles/invalidrolename", environ_overrides={'REMOTE_USER': "test"}
+        )
+        assert response.status_code == 404
+        assert response.json == {
+            'detail': "The Role with name `invalidrolename` was not found",
+            'status': 404,
+            'title': 'Role not found',
+            'type': EXCEPTIONS_LINK_MAP[404],
+        }
+
+    def test_should_raises_401_unauthenticated(self):
+        response = self.client.delete("/api/v1/roles/test")
+
+        assert_401(response)
+
+    def test_should_raise_403_forbidden(self):
+        response = self.client.delete(
+            "/api/v1/roles/test", environ_overrides={'REMOTE_USER': "test_no_permissions"}
+        )
+        assert response.status_code == 403
+
+
+class TestPatchRole(TestRoleEndpoint):
+    @parameterized.expand(
+        [
+            ({"name": "mytest"}, "mytest", []),
+            (
+                {
+                    "name": "mytest2",
+                    "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+                },
+                "mytest2",
+                [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+            ),
+        ]
+    )
+    def test_patch_should_respond_200(self, payload, expected_name, expected_actions):
+        role = create_role(self.app, 'mytestrole')
+        response = self.client.patch(
+            f"/api/v1/roles/{role.name}", json=payload, environ_overrides={'REMOTE_USER': "test"}
+        )
+        assert response.status_code == 200
+        assert response.json['name'] == expected_name
+        assert response.json["actions"] == expected_actions
+
+    @parameterized.expand(
+        [
+            (
+                "?update_mask=name",
+                {
+                    "name": "mytest2",
+                    "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+                },
+                "mytest2",
+                [],
+            ),
+            (
+                "?update_mask=name, actions",  # both name and actions in update mask
+                {
+                    "name": "mytest2",
+                    "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+                },
+                "mytest2",
+                [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+            ),
+        ]
+    )
+    def test_patch_should_respond_200_with_update_mask(
+        self, update_mask, payload, expected_name, expected_actions
+    ):
+        role = create_role(self.app, "mytestrole")
+        assert role.permissions == []
+        response = self.client.patch(
+            f"/api/v1/roles/{role.name}{update_mask}",
+            json=payload,
+            environ_overrides={'REMOTE_USER': "test"},
+        )
+        assert response.status_code == 200
+        assert response.json['name'] == expected_name
+        assert response.json['actions'] == expected_actions
+
+    def test_patch_should_respond_400_for_invalid_fields_in_update_mask(self):
+        role = create_role(self.app, "mytestrole")
+        payload = {"name": "testme"}
+        response = self.client.patch(
+            f"/api/v1/roles/{role.name}?update_mask=invalid_name",
+            json=payload,
+            environ_overrides={'REMOTE_USER': "test"},
+        )
+        assert response.status_code == 400
+        assert response.json['detail'] == "'invalid_name' in update_mask is unknown"
+
+    @parameterized.expand(
+        [
+            (
+                {
+                    "name": "testme",
+                    "permissions": [  # Using permissions instead of actions should raise
+                        {"resource": {"name": "Connections"}, "action": {"name": "can_create"}}
+                    ],
+                },
+                "{'permissions': ['Unknown field.']}",
+            ),
+            (
+                {
+                    "name": "testme",
+                    "actions": [
+                        {
+                            "view_menu": {"name": "Connections"},  # Using view_menu instead of resource
+                            "action": {"name": "can_create"},
+                        }
+                    ],
+                },
+                "{'actions': {0: {'view_menu': ['Unknown field.']}}}",
+            ),
+            (
+                {
+                    "name": "testme",
+                    "actions": [
+                        {
+                            "resource": {"name": "FooBars"},  # Using wrong resource name
+                            "action": {"name": "can_create"},
+                        }
+                    ],
+                },
+                "The specified resource: 'FooBars' was not found",
+            ),
+            (
+                {
+                    "name": "testme",
+                    "actions": [
+                        {
+                            "resource": {"name": "Connections"},  # Using wrong action name
+                            "action": {"name": "can_invalid"},
+                        }
+                    ],
+                },
+                "The specified action: 'can_invalid' was not found",
+            ),
+        ]
+    )
+    def test_patch_should_respond_400_for_invalid_update(self, payload, expected_error):
+        role = create_role(self.app, "mytestrole")
+        response = self.client.patch(
+            f"/api/v1/roles/{role.name}",
+            json=payload,
+            environ_overrides={'REMOTE_USER': "test"},
+        )
+        assert response.status_code == 400
+        assert response.json['detail'] == expected_error
+
+    def test_should_raises_401_unauthenticated(self):
+        response = self.client.patch(
+            "/api/v1/roles/test",
+            json={
+                "name": "mytest2",
+                "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+            },
+        )
+
+        assert_401(response)
+
+    def test_should_raise_403_forbidden(self):
+        response = self.client.patch(
+            "/api/v1/roles/test",
+            json={
+                "name": "mytest2",
+                "actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
+            },
+            environ_overrides={'REMOTE_USER': "test_no_permissions"},
+        )
+        assert response.status_code == 403