Skip to content

Commit

Permalink
Update Rest API tests to no longer rely on FAB auth manager. Move tes…
Browse files Browse the repository at this point in the history
…ts specific to FAB permissions to FAB provider (#42523)
  • Loading branch information
vincbeck authored Oct 1, 2024
1 parent e46365d commit 9536c98
Show file tree
Hide file tree
Showing 70 changed files with 3,208 additions and 1,542 deletions.
7 changes: 6 additions & 1 deletion airflow/auth/managers/simple/simple_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,12 @@ def _is_authorized(
user = self.get_user()
if not user:
return False
role_str = user.get_role().upper()

user_role = user.get_role()
if not user_role:
return False

role_str = user_role.upper()
role = SimpleAuthManagerRole[role_str]
if role == SimpleAuthManagerRole.ADMIN:
return True
Expand Down
6 changes: 3 additions & 3 deletions airflow/auth/managers/simple/user.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,10 @@ class SimpleAuthManagerUser(BaseUser):
User model for users managed by the simple auth manager.
:param username: The username
:param role: The role associated to the user
:param role: The role associated to the user. If not provided, the user has no permission
"""

def __init__(self, *, username: str, role: str) -> None:
def __init__(self, *, username: str, role: str | None) -> None:
self.username = username
self.role = role

Expand All @@ -37,5 +37,5 @@ def get_id(self) -> str:
def get_name(self) -> str:
return self.username

def get_role(self):
def get_role(self) -> str | None:
return self.role
52 changes: 52 additions & 0 deletions airflow/migrations/versions/0034_3_0_0_update_user_id_type.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
#
# 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.

"""
Update dag_run_note.user_id and task_instance_note.user_id columns to String.
Revision ID: 44eabb1904b4
Revises: 16cbcb1c8c36
Create Date: 2024-09-27 09:57:29.830521
"""

from __future__ import annotations

import sqlalchemy as sa
from alembic import op

# revision identifiers, used by Alembic.
revision = "44eabb1904b4"
down_revision = "16cbcb1c8c36"
branch_labels = None
depends_on = None
airflow_version = "3.0.0"


def upgrade():
with op.batch_alter_table("dag_run_note") as batch_op:
batch_op.alter_column("user_id", type_=sa.String(length=128))
with op.batch_alter_table("task_instance_note") as batch_op:
batch_op.alter_column("user_id", type_=sa.String(length=128))


def downgrade():
with op.batch_alter_table("dag_run_note") as batch_op:
batch_op.alter_column("user_id", type_=sa.Integer(), postgresql_using="user_id::integer")
with op.batch_alter_table("task_instance_note") as batch_op:
batch_op.alter_column("user_id", type_=sa.Integer(), postgresql_using="user_id::integer")
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
also rename the one on DatasetAliasModel here for consistency.
Revision ID: 0d9e73a75ee4
Revises: 16cbcb1c8c36
Revises: 44eabb1904b4
Create Date: 2024-08-13 09:45:32.213222
"""

Expand All @@ -42,7 +42,7 @@

# revision identifiers, used by Alembic.
revision = "0d9e73a75ee4"
down_revision = "16cbcb1c8c36"
down_revision = "44eabb1904b4"
branch_labels = None
depends_on = None
airflow_version = "3.0.0"
Expand Down
2 changes: 1 addition & 1 deletion airflow/models/dagrun.py
Original file line number Diff line number Diff line change
Expand Up @@ -1687,7 +1687,7 @@ class DagRunNote(Base):

__tablename__ = "dag_run_note"

user_id = Column(Integer, nullable=True)
user_id = Column(String(128), nullable=True)
dag_run_id = Column(Integer, primary_key=True, nullable=False)
content = Column(String(1000).with_variant(Text(1000), "mysql"))
created_at = Column(UtcDateTime, default=timezone.utcnow, nullable=False)
Expand Down
2 changes: 1 addition & 1 deletion airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -4002,7 +4002,7 @@ class TaskInstanceNote(TaskInstanceDependencies):

__tablename__ = "task_instance_note"

user_id = Column(Integer, nullable=True)
user_id = Column(String(128), nullable=True)
task_id = Column(StringID(), primary_key=True, nullable=False)
dag_id = Column(StringID(), primary_key=True, nullable=False)
run_id = Column(StringID(), primary_key=True, nullable=False)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,7 @@ def requires_authentication(function: T):

@wraps(function)
def decorated(*args, **kwargs):
if auth_current_user() is not None or current_app.appbuilder.get_app.config.get(
"AUTH_ROLE_PUBLIC", None
):
if auth_current_user() is not None or current_app.config.get("AUTH_ROLE_PUBLIC", None):
return function(*args, **kwargs)
else:
return Response("Unauthorized", 401, {"WWW-Authenticate": "Basic"})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ def requires_authentication(function: T, find_user: Callable[[str], BaseUser] |

@wraps(function)
def decorated(*args, **kwargs):
if current_app.appbuilder.get_app.config.get("AUTH_ROLE_PUBLIC", None):
if current_app.config.get("AUTH_ROLE_PUBLIC", None):
response = function(*args, **kwargs)
return make_response(response)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ class AnonymousUser(AnonymousUserMixin, BaseUser):
@property
def roles(self):
if not self._roles:
public_role = current_app.appbuilder.get_app.config.get("AUTH_ROLE_PUBLIC", None)
public_role = current_app.config.get("AUTH_ROLE_PUBLIC", None)
self._roles = {current_app.appbuilder.sm.find_role(public_role)} if public_role else set()
return self._roles

Expand Down
2 changes: 1 addition & 1 deletion docs/apache-airflow/img/airflow_erd.sha256
Original file line number Diff line number Diff line change
@@ -1 +1 @@
c33e9a583a5b29eb748ebd50e117643e11bcb2a9b61ec017efd690621e22769b
64dfad12dfd49f033c4723c2f3bb3bac58dd956136fb24a87a2e5a6ae176ec1a
4 changes: 2 additions & 2 deletions docs/apache-airflow/img/airflow_erd.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
5 changes: 4 additions & 1 deletion docs/apache-airflow/migrations-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,10 @@ Here's the list of all the Database Migrations that are executed via when you ru
+-------------------------+------------------+-------------------+--------------------------------------------------------------+
| Revision ID | Revises ID | Airflow Version | Description |
+=========================+==================+===================+==============================================================+
| ``0d9e73a75ee4`` (head) | ``16cbcb1c8c36`` | ``3.0.0`` | Add name and group fields to DatasetModel. |
| ``0d9e73a75ee4`` (head) | ``44eabb1904b4`` | ``3.0.0`` | Add name and group fields to DatasetModel. |
+-------------------------+------------------+-------------------+--------------------------------------------------------------+
| ``44eabb1904b4`` | ``16cbcb1c8c36`` | ``3.0.0`` | Update dag_run_note.user_id and task_instance_note.user_id |
| | | | columns to String. |
+-------------------------+------------------+-------------------+--------------------------------------------------------------+
| ``16cbcb1c8c36`` | ``522625f6d606`` | ``3.0.0`` | Remove redundant index. |
+-------------------------+------------------+-------------------+--------------------------------------------------------------+
Expand Down
11 changes: 9 additions & 2 deletions tests/api_connexion/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,16 @@ def minimal_app_for_api():
]
)
def factory():
with conf_vars({("api", "auth_backends"): "tests.test_utils.remote_user_api_auth_backend"}):
with conf_vars(
{
("api", "auth_backends"): "tests.test_utils.remote_user_api_auth_backend",
(
"core",
"auth_manager",
): "airflow.auth.managers.simple.simple_auth_manager.SimpleAuthManager",
}
):
_app = app.create_app(testing=True, config={"WTF_CSRF_ENABLED": False}) # type:ignore
_app.config["AUTH_ROLE_PUBLIC"] = None
return _app

return factory()
Expand Down
31 changes: 5 additions & 26 deletions tests/api_connexion/endpoints/test_backfill_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
from airflow.models.dag import DAG
from airflow.models.serialized_dag import SerializedDagModel
from airflow.operators.empty import EmptyOperator
from airflow.security import permissions
from airflow.utils import timezone
from airflow.utils.session import provide_session
from tests.test_utils.api_connexion_utils import create_user, delete_user
Expand All @@ -50,25 +49,11 @@ def configured_app(minimal_app_for_api):
app = minimal_app_for_api

create_user(
app, # type: ignore
app,
username="test",
role_name="Test",
permissions=[
(permissions.ACTION_CAN_READ, permissions.RESOURCE_DAG),
(permissions.ACTION_CAN_EDIT, permissions.RESOURCE_DAG),
(permissions.ACTION_CAN_DELETE, permissions.RESOURCE_DAG),
],
)
create_user(app, username="test_no_permissions", role_name="TestNoPermissions") # type: ignore
create_user(app, username="test_granular_permissions", role_name="TestGranularDag") # type: ignore
app.appbuilder.sm.sync_perm_for_dag( # type: ignore
"TEST_DAG_1",
access_control={
"TestGranularDag": {
permissions.RESOURCE_DAG: {permissions.ACTION_CAN_EDIT, permissions.ACTION_CAN_READ}
},
},
role_name="admin",
)
create_user(app, username="test_no_permissions", role_name=None)

with DAG(
DAG_ID,
Expand All @@ -93,9 +78,8 @@ def configured_app(minimal_app_for_api):

yield app

delete_user(app, username="test") # type: ignore
delete_user(app, username="test_no_permissions") # type: ignore
delete_user(app, username="test_granular_permissions") # type: ignore
delete_user(app, username="test")
delete_user(app, username="test_no_permissions")


class TestBackfillEndpoint:
Expand Down Expand Up @@ -178,7 +162,6 @@ def test_should_respond_200(self, session):
@pytest.mark.parametrize(
"user, expected",
[
("test_granular_permissions", 200),
("test_no_permissions", 403),
("test", 200),
(None, 401),
Expand Down Expand Up @@ -240,7 +223,6 @@ def test_no_exist(self, session):
@pytest.mark.parametrize(
"user, expected",
[
("test_granular_permissions", 200),
("test_no_permissions", 403),
("test", 200),
(None, 401),
Expand Down Expand Up @@ -268,7 +250,6 @@ class TestCreateBackfill(TestBackfillEndpoint):
@pytest.mark.parametrize(
"user, expected",
[
("test_granular_permissions", 200),
("test_no_permissions", 403),
("test", 200),
(None, 401),
Expand Down Expand Up @@ -347,7 +328,6 @@ def test_should_respond_200(self, session):
@pytest.mark.parametrize(
"user, expected",
[
("test_granular_permissions", 200),
("test_no_permissions", 403),
("test", 200),
(None, 401),
Expand Down Expand Up @@ -409,7 +389,6 @@ def test_should_respond_200(self, session):
@pytest.mark.parametrize(
"user, expected",
[
("test_granular_permissions", 200),
("test_no_permissions", 403),
("test", 200),
(None, 401),
Expand Down
12 changes: 5 additions & 7 deletions tests/api_connexion/endpoints/test_config_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import pytest

from airflow.security import permissions
from tests.test_utils.api_connexion_utils import assert_401, create_user, delete_user
from tests.test_utils.config import conf_vars

Expand Down Expand Up @@ -54,18 +53,17 @@
def configured_app(minimal_app_for_api):
app = minimal_app_for_api
create_user(
app, # type:ignore
app,
username="test",
role_name="Test",
permissions=[(permissions.ACTION_CAN_READ, permissions.RESOURCE_CONFIG)], # type: ignore
role_name="admin",
)
create_user(app, username="test_no_permissions", role_name="TestNoPermissions") # type: ignore
create_user(app, username="test_no_permissions", role_name=None)

with conf_vars({("webserver", "expose_config"): "True"}):
yield minimal_app_for_api

delete_user(app, username="test") # type: ignore
delete_user(app, username="test_no_permissions") # type: ignore
delete_user(app, username="test")
delete_user(app, username="test_no_permissions")


class TestGetConfig:
Expand Down
17 changes: 5 additions & 12 deletions tests/api_connexion/endpoints/test_connection_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
from airflow.api_connexion.exceptions import EXCEPTIONS_LINK_MAP
from airflow.models import Connection
from airflow.secrets.environment_variables import CONN_ENV_PREFIX
from airflow.security import permissions
from airflow.utils.session import provide_session
from tests.test_utils.api_connexion_utils import assert_401, create_user, delete_user
from tests.test_utils.config import conf_vars
Expand All @@ -38,22 +37,16 @@
def configured_app(minimal_app_for_api):
app = minimal_app_for_api
create_user(
app, # type: ignore
app,
username="test",
role_name="Test",
permissions=[
(permissions.ACTION_CAN_CREATE, permissions.RESOURCE_CONNECTION),
(permissions.ACTION_CAN_READ, permissions.RESOURCE_CONNECTION),
(permissions.ACTION_CAN_EDIT, permissions.RESOURCE_CONNECTION),
(permissions.ACTION_CAN_DELETE, permissions.RESOURCE_CONNECTION),
],
role_name="admin",
)
create_user(app, username="test_no_permissions", role_name="TestNoPermissions") # type: ignore
create_user(app, username="test_no_permissions", role_name=None)

yield app

delete_user(app, username="test") # type: ignore
delete_user(app, username="test_no_permissions") # type: ignore
delete_user(app, username="test")
delete_user(app, username="test_no_permissions")


class TestConnectionEndpoint:
Expand Down
Loading

0 comments on commit 9536c98

Please sign in to comment.