Skip to content
New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

Let auth managers provide their own API endpoints #34349

Merged
merged 25 commits into from
Oct 17, 2023
Merged
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
1472fa0
move user & role API endpoints to fab folder
vandonr-amz Sep 12, 2023
a3632f9
add a method to init API from auth provider
vandonr-amz Sep 13, 2023
7cfe915
add user & role APIs in fab auth manager
vandonr-amz Sep 13, 2023
43f3e5d
mark existing api endpoints as deprecated
vandonr-amz Sep 13, 2023
6f835d8
Merge remote-tracking branch 'apache/main' into vandonr/fab
vandonr-amz Sep 13, 2023
3c0490d
move tests
vandonr-amz Sep 13, 2023
f67eac3
test fixes
vandonr-amz Sep 13, 2023
32e67aa
apply suggestion
vandonr-amz Sep 13, 2023
8a8c738
docstring change
vandonr-amz Sep 13, 2023
8c104aa
more test fixing
vandonr-amz Sep 13, 2023
5ee06cf
change endpoint path to auth/fab
vandonr-amz Sep 22, 2023
7183e9b
Merge remote-tracking branch 'apache/main' into vandonr/fab
vandonr-amz Sep 22, 2023
51f8038
add intermediate layer on moved endpoints to check provider
vandonr-amz Sep 22, 2023
ec46600
a bit more detail in http error
vandonr-amz Sep 22, 2023
8566e29
static check fix
vandonr-amz Sep 25, 2023
73d014b
Merge remote-tracking branch 'apache/main' into vandonr/fab
vandonr-amz Sep 25, 2023
9301bb5
Merge branch 'main' into vandonr/fab
vandonr-amz Sep 25, 2023
95d57a2
Merge branch 'main' into vandonr/fab
vincbeck Sep 26, 2023
df4c63f
Add file `airflow/auth/managers/fab/openapi/v1.yaml` to MANIFEST.in
vincbeck Sep 26, 2023
1ad6988
Merge branch 'main' into vandonr/fab
vincbeck Oct 4, 2023
94ae9a2
Fix comment
vincbeck Oct 6, 2023
b89f301
Fix airflow/auth/managers/fab/openapi/v1.yaml
vincbeck Oct 6, 2023
6e6eb9c
Merge branch 'main' into vandonr/fab
vincbeck Oct 16, 2023
cc17098
Replace `get_api_blueprint` by `get_api_endpoints`
vincbeck Oct 16, 2023
d9ca5bf
Merge branch 'main' into vandonr/fab
vincbeck Oct 17, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions MANIFEST.in
Original file line number Diff line number Diff line change
@@ -32,6 +32,7 @@ exclude airflow/www/yarn.lock
exclude airflow/www/*.sh
include airflow/alembic.ini
include airflow/api_connexion/openapi/v1.yaml
include airflow/auth/managers/fab/openapi/v1.yaml
include airflow/git_version
include airflow/provider_info.schema.json
include airflow/customized_form_field_behaviours.schema.json
126 changes: 126 additions & 0 deletions airflow/api_connexion/endpoints/forward_to_fab_endpoint.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,126 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations

import warnings
from typing import TYPE_CHECKING

from airflow.api_connexion.exceptions import BadRequest
from airflow.auth.managers.fab.api_endpoints import role_and_permission_endpoint, user_endpoint
from airflow.www.extensions.init_auth_manager import get_auth_manager

if TYPE_CHECKING:
from typing import Callable

from airflow.api_connexion.types import APIResponse


def _require_fab(func: Callable) -> Callable:
"""
Raise an HTTP error 400 if the auth manager is not FAB.

Intended to decorate endpoints that have been migrated from Airflow API to FAB API.
"""

def inner(*args, **kwargs):
from airflow.auth.managers.fab.fab_auth_manager import FabAuthManager

auth_mgr = get_auth_manager()
if not isinstance(auth_mgr, FabAuthManager):
raise BadRequest(
detail="This endpoint is only available when using the default auth manager FabAuthManager."
)
else:
warnings.warn(
"This API endpoint is deprecated. "
"Please use the API under /auth/fab/v1 instead for this operation.",
DeprecationWarning,
)
return func(*args, **kwargs)

return inner


### role


@_require_fab
def get_role(**kwargs) -> APIResponse:
"""Get role."""
return role_and_permission_endpoint.get_role(**kwargs)


@_require_fab
def get_roles(**kwargs) -> APIResponse:
"""Get roles."""
return role_and_permission_endpoint.get_roles(**kwargs)


@_require_fab
def delete_role(**kwargs) -> APIResponse:
"""Delete a role."""
return role_and_permission_endpoint.delete_role(**kwargs)


@_require_fab
def patch_role(**kwargs) -> APIResponse:
"""Update a role."""
return role_and_permission_endpoint.patch_role(**kwargs)


@_require_fab
def post_role(**kwargs) -> APIResponse:
"""Create a new role."""
return role_and_permission_endpoint.post_role(**kwargs)


### permissions
@_require_fab
def get_permissions(**kwargs) -> APIResponse:
"""Get permissions."""
return role_and_permission_endpoint.get_permissions(**kwargs)


### user
@_require_fab
def get_user(**kwargs) -> APIResponse:
"""Get a user."""
return user_endpoint.get_user(**kwargs)


@_require_fab
def get_users(**kwargs) -> APIResponse:
"""Get users."""
return user_endpoint.get_users(**kwargs)


@_require_fab
def post_user(**kwargs) -> APIResponse:
"""Create a new user."""
return user_endpoint.post_user(**kwargs)


@_require_fab
def patch_user(**kwargs) -> APIResponse:
"""Update a user."""
return user_endpoint.patch_user(**kwargs)


@_require_fab
def delete_user(**kwargs) -> APIResponse:
"""Delete a user."""
return user_endpoint.delete_user(**kwargs)
33 changes: 22 additions & 11 deletions airflow/api_connexion/openapi/v1.yaml
Original file line number Diff line number Diff line change
@@ -2127,12 +2127,13 @@ paths:

/roles:
get:
deprecated: true
summary: List roles
description: |
Get a list of roles.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: get_roles
tags: [Role]
parameters:
@@ -2152,12 +2153,13 @@ paths:
$ref: '#/components/responses/PermissionDenied'

post:
deprecated: true
summary: Create a role
description: |
Create a new role.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: post_role
tags: [Role]
requestBody:
@@ -2185,12 +2187,13 @@ paths:
- $ref: '#/components/parameters/RoleName'

get:
deprecated: true
summary: Get a role
description: |
Get a role.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: get_role
tags: [Role]
responses:
@@ -2208,12 +2211,13 @@ paths:
$ref: '#/components/responses/NotFound'

patch:
deprecated: true
summary: Update a role
description: |
Update a role.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: patch_role
tags: [Role]
parameters:
@@ -2242,12 +2246,13 @@ paths:
$ref: '#/components/responses/NotFound'

delete:
deprecated: true
summary: Delete a role
description: |
Delete a role.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: delete_role
tags: [Role]
responses:
@@ -2264,12 +2269,13 @@ paths:

/permissions:
get:
deprecated: true
summary: List permissions
description: |
Get a list of permissions.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.role_and_permission_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: get_permissions
tags: [Permission]
parameters:
@@ -2289,12 +2295,13 @@ paths:

/users:
get:
deprecated: true
summary: List users
description: |
Get a list of users.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.user_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: get_users
tags: [User]
parameters:
@@ -2314,12 +2321,13 @@ paths:
$ref: '#/components/responses/PermissionDenied'

post:
deprecated: true
summary: Create a user
description: |
Create a new user with unique username and email.

*New in version 2.2.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.user_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: post_user
tags: [User]
requestBody:
@@ -2348,12 +2356,13 @@ paths:
parameters:
- $ref: '#/components/parameters/Username'
get:
deprecated: true
summary: Get a user
description: |
Get a user with a specific username.

*New in version 2.1.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.user_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: get_user
tags: [User]
responses:
@@ -2371,12 +2380,13 @@ paths:
$ref: '#/components/responses/NotFound'

patch:
deprecated: true
summary: Update a user
description: |
Update fields for a user.

*New in version 2.2.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.user_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: patch_user
tags: [User]
parameters:
@@ -2404,12 +2414,13 @@ paths:
$ref: '#/components/responses/NotFound'

delete:
deprecated: true
summary: Delete a user
description: |
Delete a user with a specific username.

*New in version 2.2.0*
x-openapi-router-controller: airflow.api_connexion.endpoints.user_endpoint
x-openapi-router-controller: airflow.api_connexion.endpoints.forward_to_fab_endpoint
operationId: delete_user
tags: [User]
responses:
5 changes: 5 additions & 0 deletions airflow/auth/managers/base_auth_manager.py
Original file line number Diff line number Diff line change
@@ -28,6 +28,7 @@
from airflow.utils.session import NEW_SESSION, provide_session

if TYPE_CHECKING:
from connexion import FlaskApi
from flask import Flask
from sqlalchemy.orm import Session

@@ -66,6 +67,10 @@ def get_cli_commands() -> list[CLICommand]:
"""
return []

def get_api_endpoints(self) -> None | FlaskApi:
"""Return API endpoint(s) definition for the auth manager."""
return None

@abstractmethod
def get_user_name(self) -> str:
"""Return the username associated to the user in session."""
16 changes: 16 additions & 0 deletions airflow/auth/managers/fab/api_endpoints/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
22 changes: 21 additions & 1 deletion airflow/auth/managers/fab/fab_auth_manager.py
Original file line number Diff line number Diff line change
@@ -18,8 +18,10 @@
from __future__ import annotations

import warnings
from pathlib import Path
from typing import TYPE_CHECKING, Container

from connexion import FlaskApi
from flask import url_for
from sqlalchemy import select
from sqlalchemy.orm import Session, joinedload
@@ -43,6 +45,7 @@
from airflow.cli.cli_config import (
GroupCommand,
)
from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.models import DagModel
from airflow.security import permissions
@@ -75,9 +78,10 @@
RESOURCE_XCOM,
)
from airflow.utils.session import NEW_SESSION, provide_session
from airflow.utils.yaml import safe_load
from airflow.www.extensions.init_views import _CustomErrorRequestBodyValidator, _LazyResolver

if TYPE_CHECKING:

from airflow.auth.managers.models.base_user import BaseUser
from airflow.cli.cli_config import (
CLICommand,
@@ -133,6 +137,22 @@ def get_cli_commands() -> list[CLICommand]:
SYNC_PERM_COMMAND, # not in a command group
]

def get_api_endpoints(self) -> None | FlaskApi:
folder = Path(__file__).parents[0].resolve() # this is airflow/auth/managers/fab/
with folder.joinpath("openapi", "v1.yaml").open() as f:
specification = safe_load(f)
return FlaskApi(
specification=specification,
resolver=_LazyResolver(),
base_path="/auth/fab/v1",
options={
"swagger_ui": conf.getboolean("webserver", "enable_swagger_ui", fallback=True),
},
strict_validation=True,
validate_responses=True,
validator_map={"body": _CustomErrorRequestBodyValidator},
)

def get_user_display_name(self) -> str:
"""Return the user's display name associated to the user in session."""
user = self.get_user()
700 changes: 700 additions & 0 deletions airflow/auth/managers/fab/openapi/v1.yaml

Large diffs are not rendered by default.

4 changes: 4 additions & 0 deletions airflow/www/app.py
Original file line number Diff line number Diff line change
@@ -48,7 +48,9 @@
)
from airflow.www.extensions.init_session import init_airflow_session_interface
from airflow.www.extensions.init_views import (
init_api_auth_provider,
init_api_connexion,
init_api_error_handlers,
init_api_experimental,
init_api_internal,
init_appbuilder_views,
@@ -169,6 +171,8 @@ def create_app(config=None, testing=False):
raise RuntimeError("The AIP_44 is not enabled so you cannot use it.")
init_api_internal(flask_app)
init_api_experimental(flask_app)
init_api_auth_provider(flask_app)
init_api_error_handlers(flask_app) # needs to be after all api inits to let them add their path first

sync_appbuilder_roles(flask_app)

34 changes: 28 additions & 6 deletions airflow/www/extensions/init_views.py
Original file line number Diff line number Diff line change
@@ -33,6 +33,7 @@
from airflow.exceptions import RemovedInAirflow3Warning
from airflow.security import permissions
from airflow.utils.yaml import safe_load
from airflow.www.extensions.init_auth_manager import get_auth_manager

if TYPE_CHECKING:
from flask import Flask
@@ -230,15 +231,16 @@ def validate_schema(self, data, url):
return super().validate_schema(data, url)


def init_api_connexion(app: Flask) -> None:
"""Initialize Stable API."""
base_path = "/api/v1"
base_paths: list[str] = [] # contains the list of base paths that have api endpoints


def init_api_error_handlers(app: Flask) -> None:
"""Add error handlers for 404 and 405 errors for existing API paths."""
from airflow.www import views

@app.errorhandler(404)
def _handle_api_not_found(ex):
if request.path.startswith(base_path):
if any([request.path.startswith(p) for p in base_paths]):
# 404 errors are never handled on the blueprint level
# unless raised from a view func so actual 404 errors,
# i.e. "no route for it" defined, need to be handled
@@ -249,11 +251,19 @@ def _handle_api_not_found(ex):

@app.errorhandler(405)
def _handle_method_not_allowed(ex):
if request.path.startswith(base_path):
if any([request.path.startswith(p) for p in base_paths]):
return common_error_handler(ex)
else:
return views.method_not_allowed(ex)

app.register_error_handler(ProblemException, common_error_handler)


def init_api_connexion(app: Flask) -> None:
"""Initialize Stable API."""
base_path = "/api/v1"
base_paths.append(base_path)

with ROOT_APP_DIR.joinpath("api_connexion", "openapi", "v1.yaml").open() as f:
specification = safe_load(f)
api_bp = FlaskApi(
@@ -271,7 +281,6 @@ def _handle_method_not_allowed(ex):
api_bp.after_request(set_cors_headers_on_response)

app.register_blueprint(api_bp)
app.register_error_handler(ProblemException, common_error_handler)
app.extensions["csrf"].exempt(api_bp)


@@ -280,6 +289,7 @@ def init_api_internal(app: Flask, standalone_api: bool = False) -> None:
if not standalone_api and not conf.getboolean("webserver", "run_internal_api", fallback=False):
return

base_paths.append("/internal_api/v1")
with ROOT_APP_DIR.joinpath("api_internal", "openapi", "internal_api_v1.yaml").open() as f:
specification = safe_load(f)
api_bp = FlaskApi(
@@ -308,5 +318,17 @@ def init_api_experimental(app):
"The authenticated user has full access.",
RemovedInAirflow3Warning,
)
base_paths.append("/api/experimental")
app.register_blueprint(endpoints.api_experimental, url_prefix="/api/experimental")
app.extensions["csrf"].exempt(endpoints.api_experimental)


def init_api_auth_provider(app):
"""Initialize the API offered by the auth manager."""
auth_mgr = get_auth_manager()
api = auth_mgr.get_api_endpoints()
if api:
blueprint = api.blueprint
base_paths.append(blueprint.url_prefix)
app.register_blueprint(blueprint)
app.extensions["csrf"].exempt(blueprint)
1 change: 1 addition & 0 deletions setup.cfg
Original file line number Diff line number Diff line change
@@ -172,6 +172,7 @@ airflow=
provider_info.schema.json

airflow.api_connexion.openapi=*.yaml
airflow.auth.managers.fab.openapi=*.yaml
airflow.serialization=*.json
airflow.utils=
context.pyi
1 change: 1 addition & 0 deletions tests/api_connexion/conftest.py
Original file line number Diff line number Diff line change
@@ -33,6 +33,7 @@ def minimal_app_for_api():
"init_appbuilder",
"init_api_experimental_auth",
"init_api_connexion",
"init_api_error_handlers",
"init_airflow_session_interface",
"init_appbuilder_views",
]
16 changes: 16 additions & 0 deletions tests/auth/managers/fab/api_endpoints/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
40 changes: 40 additions & 0 deletions tests/auth/managers/fab/api_endpoints/conftest.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from __future__ import annotations

import pytest

from airflow.www import app
from tests.test_utils.config import conf_vars
from tests.test_utils.decorators import dont_initialize_flask_app_submodules


@pytest.fixture(scope="session")
def minimal_app_for_auth_api():
@dont_initialize_flask_app_submodules(
skip_all_except=[
"init_appbuilder",
"init_api_experimental_auth",
"init_api_auth_provider",
"init_api_error_handlers",
]
)
def factory():
with conf_vars({("api", "auth_backends"): "tests.test_utils.remote_user_api_auth_backend"}):
return app.create_app(testing=True, config={"WTF_CSRF_ENABLED": False}) # type:ignore

return factory()
Original file line number Diff line number Diff line change
@@ -32,8 +32,8 @@


@pytest.fixture(scope="module")
def configured_app(minimal_app_for_api):
app = minimal_app_for_api
def configured_app(minimal_app_for_auth_api):
app = minimal_app_for_auth_api
create_user(
app, # type: ignore
username="test",
@@ -74,12 +74,14 @@ def teardown_method(self):

class TestGetRoleEndpoint(TestRoleEndpoint):
def test_should_response_200(self):
response = self.client.get("/api/v1/roles/Admin", environ_overrides={"REMOTE_USER": "test"})
response = self.client.get("/auth/fab/v1/roles/Admin", environ_overrides={"REMOTE_USER": "test"})
assert response.status_code == 200
assert response.json["name"] == "Admin"

def test_should_respond_404(self):
response = self.client.get("/api/v1/roles/invalid-role", environ_overrides={"REMOTE_USER": "test"})
response = self.client.get(
"/auth/fab/v1/roles/invalid-role", environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 404
assert {
"detail": "Role with name 'invalid-role' was not found",
@@ -89,19 +91,19 @@ def test_should_respond_404(self):
} == response.json

def test_should_raises_401_unauthenticated(self):
response = self.client.get("/api/v1/roles/Admin")
response = self.client.get("/auth/fab/v1/roles/Admin")
assert_401(response)

def test_should_raise_403_forbidden(self):
response = self.client.get(
"/api/v1/roles/Admin", environ_overrides={"REMOTE_USER": "test_no_permissions"}
"/auth/fab/v1/roles/Admin", environ_overrides={"REMOTE_USER": "test_no_permissions"}
)
assert response.status_code == 403


class TestGetRolesEndpoint(TestRoleEndpoint):
def test_should_response_200(self):
response = self.client.get("/api/v1/roles", environ_overrides={"REMOTE_USER": "test"})
response = self.client.get("/auth/fab/v1/roles", environ_overrides={"REMOTE_USER": "test"})
assert response.status_code == 200
existing_roles = set(EXISTING_ROLES)
existing_roles.update(["Test", "TestNoPermissions"])
@@ -110,40 +112,42 @@ def test_should_response_200(self):
assert roles == existing_roles

def test_should_raises_401_unauthenticated(self):
response = self.client.get("/api/v1/roles")
response = self.client.get("/auth/fab/v1/roles")
assert_401(response)

def test_should_raises_400_for_invalid_order_by(self):
response = self.client.get(
"/api/v1/roles?order_by=invalid", environ_overrides={"REMOTE_USER": "test"}
"/auth/fab/v1/roles?order_by=invalid", environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 400
msg = "Ordering with 'invalid' is disallowed or the attribute does not exist on the model"
assert response.json["detail"] == msg

def test_should_raise_403_forbidden(self):
response = self.client.get("/api/v1/roles", environ_overrides={"REMOTE_USER": "test_no_permissions"})
response = self.client.get(
"/auth/fab/v1/roles", environ_overrides={"REMOTE_USER": "test_no_permissions"}
)
assert response.status_code == 403


class TestGetRolesEndpointPaginationandFilter(TestRoleEndpoint):
@pytest.mark.parametrize(
"url, expected_roles",
[
("/api/v1/roles?limit=1", ["Admin"]),
("/api/v1/roles?limit=2", ["Admin", "Op"]),
("/auth/fab/v1/roles?limit=1", ["Admin"]),
("/auth/fab/v1/roles?limit=2", ["Admin", "Op"]),
(
"/api/v1/roles?offset=1",
"/auth/fab/v1/roles?offset=1",
["Op", "Public", "Test", "TestNoPermissions", "User", "Viewer"],
),
(
"/api/v1/roles?offset=0",
"/auth/fab/v1/roles?offset=0",
["Admin", "Op", "Public", "Test", "TestNoPermissions", "User", "Viewer"],
),
("/api/v1/roles?limit=1&offset=2", ["Public"]),
("/api/v1/roles?limit=1&offset=1", ["Op"]),
("/auth/fab/v1/roles?limit=1&offset=2", ["Public"]),
("/auth/fab/v1/roles?limit=1&offset=1", ["Op"]),
(
"/api/v1/roles?limit=2&offset=2",
"/auth/fab/v1/roles?limit=2&offset=2",
["Public", "Test"],
),
],
@@ -161,20 +165,20 @@ def test_can_handle_limit_and_offset(self, url, expected_roles):

class TestGetPermissionsEndpoint(TestRoleEndpoint):
def test_should_response_200(self):
response = self.client.get("/api/v1/permissions", environ_overrides={"REMOTE_USER": "test"})
response = self.client.get("/auth/fab/v1/permissions", environ_overrides={"REMOTE_USER": "test"})
actions = {i[0] for i in self.app.appbuilder.sm.get_all_permissions() if i}
assert response.status_code == 200
assert response.json["total_entries"] == len(actions)
returned_actions = {perm["name"] for perm in response.json["actions"]}
assert actions == returned_actions

def test_should_raises_401_unauthenticated(self):
response = self.client.get("/api/v1/permissions")
response = self.client.get("/auth/fab/v1/permissions")
assert_401(response)

def test_should_raise_403_forbidden(self):
response = self.client.get(
"/api/v1/permissions", environ_overrides={"REMOTE_USER": "test_no_permissions"}
"/auth/fab/v1/permissions", environ_overrides={"REMOTE_USER": "test_no_permissions"}
)
assert response.status_code == 403

@@ -185,7 +189,9 @@ def test_post_should_respond_200(self):
"name": "Test2",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
}
response = self.client.post("/api/v1/roles", json=payload, environ_overrides={"REMOTE_USER": "test"})
response = self.client.post(
"/auth/fab/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
@@ -256,7 +262,9 @@ def test_post_should_respond_200(self):
],
)
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"})
response = self.client.post(
"/auth/fab/v1/roles", json=payload, environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 400
assert response.json == {
"detail": error_message,
@@ -270,7 +278,9 @@ def test_post_should_respond_409_already_exist(self):
"name": "Test",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
}
response = self.client.post("/api/v1/roles", json=payload, environ_overrides={"REMOTE_USER": "test"})
response = self.client.post(
"/auth/fab/v1/roles", json=payload, environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 409
assert response.json == {
"detail": "Role with name 'Test' already exists; please update with the PATCH endpoint",
@@ -281,7 +291,7 @@ def test_post_should_respond_409_already_exist(self):

def test_should_raises_401_unauthenticated(self):
response = self.client.post(
"/api/v1/roles",
"/auth/fab/v1/roles",
json={
"name": "Test2",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
@@ -292,7 +302,7 @@ def test_should_raises_401_unauthenticated(self):

def test_should_raise_403_forbidden(self):
response = self.client.post(
"/api/v1/roles",
"/auth/fab/v1/roles",
json={
"name": "mytest2",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
@@ -305,14 +315,16 @@ def test_should_raise_403_forbidden(self):
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"})
response = self.client.delete(
f"/auth/fab/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"}
"/auth/fab/v1/roles/invalidrolename", environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 404
assert response.json == {
@@ -323,13 +335,13 @@ def test_delete_should_respond_404(self):
}

def test_should_raises_401_unauthenticated(self):
response = self.client.delete("/api/v1/roles/test")
response = self.client.delete("/auth/fab/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"}
"/auth/fab/v1/roles/test", environ_overrides={"REMOTE_USER": "test_no_permissions"}
)
assert response.status_code == 403

@@ -352,7 +364,7 @@ class TestPatchRole(TestRoleEndpoint):
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"}
f"/auth/fab/v1/roles/{role.name}", json=payload, environ_overrides={"REMOTE_USER": "test"}
)
assert response.status_code == 200
assert response.json["name"] == expected_name
@@ -363,7 +375,7 @@ def test_patch_should_update_correct_roles_permissions(self):
create_role(self.app, "already_exists")

response = self.client.patch(
"/api/v1/roles/role_to_change",
"/auth/fab/v1/roles/role_to_change",
json={
"name": "already_exists",
"actions": [{"action": {"name": "can_delete"}, "resource": {"name": "XComs"}}],
@@ -408,7 +420,7 @@ def test_patch_should_respond_200_with_update_mask(
role = create_role(self.app, "mytestrole")
assert role.permissions == []
response = self.client.patch(
f"/api/v1/roles/{role.name}{update_mask}",
f"/auth/fab/v1/roles/{role.name}{update_mask}",
json=payload,
environ_overrides={"REMOTE_USER": "test"},
)
@@ -420,7 +432,7 @@ 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",
f"/auth/fab/v1/roles/{role.name}?update_mask=invalid_name",
json=payload,
environ_overrides={"REMOTE_USER": "test"},
)
@@ -480,7 +492,7 @@ def test_patch_should_respond_400_for_invalid_fields_in_update_mask(self):
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}",
f"/auth/fab/v1/roles/{role.name}",
json=payload,
environ_overrides={"REMOTE_USER": "test"},
)
@@ -489,7 +501,7 @@ def test_patch_should_respond_400_for_invalid_update(self, payload, expected_err

def test_should_raises_401_unauthenticated(self):
response = self.client.patch(
"/api/v1/roles/test",
"/auth/fab/v1/roles/test",
json={
"name": "mytest2",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],
@@ -500,7 +512,7 @@ def test_should_raises_401_unauthenticated(self):

def test_should_raise_403_forbidden(self):
response = self.client.patch(
"/api/v1/roles/test",
"/auth/fab/v1/roles/test",
json={
"name": "mytest2",
"actions": [{"resource": {"name": "Connections"}, "action": {"name": "can_create"}}],

Large diffs are not rendered by default.

2 changes: 2 additions & 0 deletions tests/test_utils/decorators.py
Original file line number Diff line number Diff line change
@@ -40,6 +40,8 @@ def no_op(*args, **kwargs):
"init_api_connexion",
"init_api_internal",
"init_api_experimental",
"init_api_auth_provider",
"init_api_error_handlers",
"sync_appbuilder_roles",
"init_jinja_globals",
"init_xframe_protection",