Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
61 changes: 0 additions & 61 deletions airflow/auth/managers/base_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
from airflow.auth.managers.models.base_user import BaseUser
from airflow.auth.managers.models.resource_details import DagDetails
from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.models import DagModel
from airflow.typing_compat import Literal
from airflow.utils.jwt_signer import JWTSigner, get_signing_key
Expand All @@ -38,8 +37,6 @@
from collections.abc import Container, Sequence

from fastapi import FastAPI
from flask import Blueprint
from flask_appbuilder.menu import MenuItem
from sqlalchemy.orm import Session

from airflow.auth.managers.models.batch_apis import (
Expand Down Expand Up @@ -81,22 +78,6 @@ def init(self) -> None:
By default, do nothing.
"""

def get_user_name(self) -> str:
"""Return the username associated to the user in session."""
user = self.get_user()
if not user:
self.log.error("Calling 'get_user_name()' but the user is not signed in.")
raise AirflowException("The user must be signed in.")
return user.get_name()

def get_user_display_name(self) -> str:
"""Return the user's display name associated to the user in session."""
return self.get_user_name()

@abstractmethod
def get_user(self) -> T | None:
"""Return the user associated to the user in session."""

@abstractmethod
def deserialize_user(self, token: dict[str, Any]) -> T:
"""Create a user object from dict."""
Expand All @@ -122,36 +103,10 @@ def get_jwt_token(
expiration_time_in_seconds=expiration_time_in_seconds
).generate_signed_token(self.serialize_user(user))

def get_user_id(self) -> str | None:
"""Return the user ID associated to the user in session."""
user = self.get_user()
if not user:
self.log.error("Calling 'get_user_id()' but the user is not signed in.")
raise AirflowException("The user must be signed in.")
if user_id := user.get_id():
return str(user_id)
return None

@abstractmethod
def is_logged_in(self) -> bool:
"""Return whether the user is logged in."""

@abstractmethod
def get_url_login(self, **kwargs) -> str:
"""Return the login page url."""

@abstractmethod
def get_url_logout(self) -> str:
"""Return the logout page url."""

def get_url_user_profile(self) -> str | None:
"""
Return the url to a page displaying info about the current user.

By default, return None.
"""
return None

@abstractmethod
def is_authorized_configuration(
self,
Expand Down Expand Up @@ -282,14 +237,6 @@ def is_authorized_custom_view(self, *, method: ResourceMethod | str, resource_na
:param user: the user to performing the action
"""

@abstractmethod
def filter_permitted_menu_items(self, menu_items: list[MenuItem]) -> list[MenuItem]:
"""
Filter menu items based on user permissions.

:param menu_items: list of all menu items
"""

def batch_is_authorized_connection(
self,
requests: Sequence[IsAuthorizedConnectionRequest],
Expand Down Expand Up @@ -444,11 +391,6 @@ def get_cli_commands() -> list[CLICommand]:
"""
return []

def get_api_endpoints(self) -> None | Blueprint:
"""Return API endpoint(s) definition for the auth manager."""
# TODO: Remove this method when legacy Airflow 2 UI is gone
return None

def get_fastapi_app(self) -> FastAPI | None:
"""
Specify a sub FastAPI application specific to the auth manager.
Expand All @@ -457,9 +399,6 @@ def get_fastapi_app(self) -> FastAPI | None:
"""
return None

def register_views(self) -> None:
"""Register views specific to the auth manager."""

@staticmethod
def _get_token_signer(
expiration_time_in_seconds: int = conf.getint("api", "auth_jwt_expiration_time"),
Expand Down
23 changes: 0 additions & 23 deletions airflow/auth/managers/simple/simple_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
from typing import TYPE_CHECKING, Any

from fastapi import FastAPI
from flask import session
from starlette.requests import Request
from starlette.responses import HTMLResponse
from starlette.staticfiles import StaticFiles
Expand All @@ -39,8 +38,6 @@
from airflow.settings import AIRFLOW_PATH

if TYPE_CHECKING:
from flask_appbuilder.menu import MenuItem

from airflow.auth.managers.base_auth_manager import ResourceMethod
from airflow.auth.managers.models.resource_details import (
AccessView,
Expand Down Expand Up @@ -127,27 +124,10 @@ def init(self) -> None:
with open(self.get_generated_password_file(), "w") as file:
file.write(json.dumps(passwords))

def is_logged_in(self) -> bool:
# Remove this method when legacy UI is removed
return "user" in session or conf.getboolean("core", "simple_auth_manager_all_admins")

def get_url_login(self, **kwargs) -> str:
"""Return the login page url."""
return "/auth/webapp/login"

def get_url_logout(self) -> str:
# Remove this method when legacy UI is removed
raise NotImplementedError()

def get_user(self) -> SimpleAuthManagerUser | None:
# Remove this method when legacy UI is removed
if not self.is_logged_in():
return None
if conf.getboolean("core", "simple_auth_manager_all_admins"):
return SimpleAuthManagerUser(username="anonymous", role="admin")
else:
return session["user"]

def deserialize_user(self, token: dict[str, Any]) -> SimpleAuthManagerUser:
return SimpleAuthManagerUser(username=token["username"], role=token["role"])

Expand Down Expand Up @@ -232,9 +212,6 @@ def is_authorized_custom_view(
):
return self._is_authorized(method="GET", allow_role=SimpleAuthManagerRole.VIEWER, user=user)

def filter_permitted_menu_items(self, menu_items: list[MenuItem]) -> list[MenuItem]:
return menu_items

def get_fastapi_app(self) -> FastAPI | None:
"""
Specify a sub FastAPI application specific to the auth manager.
Expand Down
15 changes: 7 additions & 8 deletions dev/breeze/tests/test_selective_checks.py
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
pytest.param(
("airflow/api/file.py",),
{
"selected-providers-list-as-string": "amazon common.compat databricks edge fab",
"selected-providers-list-as-string": "common.compat databricks edge fab",
"all-python-versions": "['3.9']",
"all-python-versions-list-as-string": "3.9",
"python-versions": "['3.9']",
Expand All @@ -162,14 +162,13 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"prod-image-build": "false",
"needs-helm-tests": "false",
"run-tests": "true",
"run-amazon-tests": "true",
"docs-build": "true",
"skip-pre-commits": "check-provider-yaml-valid,identity,lint-helm-chart,mypy-airflow,mypy-dev,"
"mypy-docs,mypy-providers,mypy-task-sdk,ts-compile-format-lint-ui",
"upgrade-to-newer-dependencies": "false",
"core-test-types-list-as-string": "API Always",
"providers-test-types-list-as-string": "Providers[amazon] Providers[common.compat,databricks,edge,fab]",
"individual-providers-test-types-list-as-string": "Providers[amazon] Providers[common.compat] Providers[databricks] Providers[edge] Providers[fab]",
"providers-test-types-list-as-string": "Providers[common.compat,databricks,edge,fab]",
"individual-providers-test-types-list-as-string": "Providers[common.compat] Providers[databricks] Providers[edge] Providers[fab]",
"testable-core-integrations": "['celery', 'kerberos']",
"testable-providers-integrations": "['cassandra', 'drill', 'kafka', 'mongo', 'pinot', 'qdrant', 'redis', 'trino', 'ydb']",
"needs-mypy": "true",
Expand Down Expand Up @@ -722,7 +721,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
("providers/amazon/src/airflow/providers/amazon/__init__.py",),
{
"selected-providers-list-as-string": "amazon apache.hive cncf.kubernetes "
"common.compat common.messaging common.sql exasol fab ftp google http imap microsoft.azure "
"common.compat common.messaging common.sql exasol ftp google http imap microsoft.azure "
"mongo mysql openlineage postgres salesforce ssh teradata",
"all-python-versions": "['3.9']",
"all-python-versions-list-as-string": "3.9",
Expand All @@ -739,7 +738,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"upgrade-to-newer-dependencies": "false",
"run-amazon-tests": "true",
"core-test-types-list-as-string": "Always",
"providers-test-types-list-as-string": "Providers[amazon] Providers[apache.hive,cncf.kubernetes,common.compat,common.messaging,common.sql,exasol,fab,ftp,http,imap,microsoft.azure,mongo,mysql,openlineage,postgres,salesforce,ssh,teradata] Providers[google]",
"providers-test-types-list-as-string": "Providers[amazon] Providers[apache.hive,cncf.kubernetes,common.compat,common.messaging,common.sql,exasol,ftp,http,imap,microsoft.azure,mongo,mysql,openlineage,postgres,salesforce,ssh,teradata] Providers[google]",
"needs-mypy": "true",
"mypy-checks": "['mypy-providers']",
},
Expand Down Expand Up @@ -774,7 +773,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
("providers/amazon/src/airflow/providers/amazon/file.py",),
{
"selected-providers-list-as-string": "amazon apache.hive cncf.kubernetes "
"common.compat common.messaging common.sql exasol fab ftp google http imap microsoft.azure "
"common.compat common.messaging common.sql exasol ftp google http imap microsoft.azure "
"mongo mysql openlineage postgres salesforce ssh teradata",
"all-python-versions": "['3.9']",
"all-python-versions-list-as-string": "3.9",
Expand All @@ -791,7 +790,7 @@ def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
"run-kubernetes-tests": "false",
"upgrade-to-newer-dependencies": "false",
"core-test-types-list-as-string": "Always",
"providers-test-types-list-as-string": "Providers[amazon] Providers[apache.hive,cncf.kubernetes,common.compat,common.messaging,common.sql,exasol,fab,ftp,http,imap,microsoft.azure,mongo,mysql,openlineage,postgres,salesforce,ssh,teradata] Providers[google]",
"providers-test-types-list-as-string": "Providers[amazon] Providers[apache.hive,cncf.kubernetes,common.compat,common.messaging,common.sql,exasol,ftp,http,imap,microsoft.azure,mongo,mysql,openlineage,postgres,salesforce,ssh,teradata] Providers[google]",
"needs-mypy": "true",
"mypy-checks": "['mypy-providers']",
},
Expand Down
12 changes: 6 additions & 6 deletions docs/apache-airflow/core-concepts/auth-manager/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -96,10 +96,8 @@ Some reasons you may want to write a custom auth manager include:
Authentication related BaseAuthManager methods
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

* ``is_logged_in``: Return whether the user is signed-in.
* ``get_user``: Return the signed-in user.
* ``get_url_login``: Return the URL the user is redirected to for signing in.
* ``get_url_logout``: Return the URL the user is redirected to for signing out.

Authorization related BaseAuthManager methods
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Expand Down Expand Up @@ -142,7 +140,6 @@ The following methods aren't required to override to have a functional Airflow a
* ``batch_is_authorized_pool``: Batch version of ``is_authorized_pool``. If not overridden, it will call ``is_authorized_pool`` for every single item.
* ``batch_is_authorized_variable``: Batch version of ``is_authorized_variable``. If not overridden, it will call ``is_authorized_variable`` for every single item.
* ``get_permitted_dag_ids``: Return the list of DAG IDs the user has access to. If not overridden, it will call ``is_authorized_dag`` for every single DAG available in the environment.
* ``filter_permitted_menu_items``: Return the menu items the user has access to. If not overridden, it will call ``has_access`` in :class:`~airflow.www.security_manager.AirflowSecurityManagerV2` for every single menu item.

CLI
^^^
Expand Down Expand Up @@ -176,10 +173,13 @@ Auth managers may vend CLI commands which will be included in the ``airflow`` co
.. note::
When creating a new auth manager, or updating any existing auth manager, be sure to not import or execute any expensive operations/code at the module level. Auth manager classes are imported in several places and if they are slow to import this will negatively impact the performance of your Airflow environment, especially for CLI commands.

Rest API
^^^^^^^^
Extending API server application
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^

Auth managers may vend Rest API endpoints which will be included in the :doc:`/stable-rest-api-ref` by implementing the ``get_api_endpoints`` method. The endpoints can be used to manage resources such as users, groups, roles (if any) handled by your auth manager. Endpoints are only vended for the currently configured auth manager.
Auth managers have the option to extend the Airflow API server. Doing so, allow, for instance, to vend additional public API endpoints.
To extend the API server application, you need to implement the ``get_fastapi_app`` method.
Such additional endpoints can be used to manage resources such as users, groups, roles (if any) handled by your auth manager.
Endpoints defined by ``get_fastapi_app`` are mounted in ``/auth``.

Next Steps
^^^^^^^^^^
Expand Down
1 change: 0 additions & 1 deletion generated/provider_dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@
"common.compat",
"common.sql",
"exasol",
"fab",
"ftp",
"google",
"http",
Expand Down
12 changes: 11 additions & 1 deletion newsfragments/aip-79.significant.rst
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,17 @@ As part of this change the following breaking changes have occurred:

- The property ``security_manager`` has been removed from the interface

- The method ``filter_permitted_menu_items`` is now abstract and must be implemented
- All these methods have been removed from the interface:

- ``filter_permitted_menu_items``
- ``get_user_name``
- ``get_user_display_name``
- ``get_user``
- ``get_user_id``
- ``is_logged_in``
- ``get_url_logout``
- ``get_api_endpoints``
- ``register_views``

- All the following method signatures changed to make the parameter ``user`` required (it was optional)

Expand Down
1 change: 0 additions & 1 deletion providers/amazon/README.rst
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ Dependent package
`apache-airflow-providers-common-compat <https://airflow.apache.org/docs/apache-airflow-providers-common-compat>`_ ``common.compat``
`apache-airflow-providers-common-sql <https://airflow.apache.org/docs/apache-airflow-providers-common-sql>`_ ``common.sql``
`apache-airflow-providers-exasol <https://airflow.apache.org/docs/apache-airflow-providers-exasol>`_ ``exasol``
`apache-airflow-providers-fab <https://airflow.apache.org/docs/apache-airflow-providers-fab>`_ ``fab``
`apache-airflow-providers-ftp <https://airflow.apache.org/docs/apache-airflow-providers-ftp>`_ ``ftp``
`apache-airflow-providers-google <https://airflow.apache.org/docs/apache-airflow-providers-google>`_ ``google``
`apache-airflow-providers-http <https://airflow.apache.org/docs/apache-airflow-providers-http>`_ ``http``
Expand Down
Loading
Loading