Skip to content
Closed
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
62 changes: 4 additions & 58 deletions airflow-core/src/airflow/api_fastapi/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,13 @@

import logging
from contextlib import AsyncExitStack, asynccontextmanager
from typing import TYPE_CHECKING, cast
from typing import cast
from urllib.parse import urlsplit

from fastapi import FastAPI
from starlette.routing import Mount

from airflow.api_fastapi.common.auth_manager import init_auth_manager
from airflow.api_fastapi.common.dagbag import create_dag_bag
from airflow.api_fastapi.core_api.app import (
init_config,
Expand All @@ -34,12 +35,8 @@
)
from airflow.api_fastapi.execution_api.app import create_task_execution_api_app
from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException
from airflow.utils.providers_configuration_loader import providers_configuration_loaded

if TYPE_CHECKING:
from airflow.api_fastapi.auth.managers.base_auth_manager import BaseAuthManager

API_BASE_URL = conf.get("api", "base_url", fallback="")
if not API_BASE_URL or not API_BASE_URL.endswith("/"):
API_BASE_URL += "/"
Expand All @@ -51,7 +48,6 @@
log = logging.getLogger(__name__)

app: FastAPI | None = None
auth_manager: BaseAuthManager | None = None


@asynccontextmanager
Expand Down Expand Up @@ -111,59 +107,9 @@ def cached_app(config=None, testing=False, apps="all") -> FastAPI:


def purge_cached_app() -> None:
"""Remove the cached version of the app and auth_manager in global state."""
global app, auth_manager
"""Remove the cached version of the app in global state."""
global app
app = None
auth_manager = None


def get_auth_manager_cls() -> type[BaseAuthManager]:
"""
Return just the auth manager class without initializing it.

Useful to save execution time if only static methods need to be called.
"""
auth_manager_cls = conf.getimport(section="core", key="auth_manager")

if not auth_manager_cls:
raise AirflowConfigException(
"No auth manager defined in the config. Please specify one using section/key [core/auth_manager]."
)

return auth_manager_cls


def create_auth_manager() -> BaseAuthManager:
"""Create the auth manager."""
global auth_manager
auth_manager_cls = get_auth_manager_cls()
auth_manager = auth_manager_cls()
return auth_manager


def init_auth_manager(app: FastAPI | None = None) -> BaseAuthManager:
"""Initialize the auth manager."""
am = create_auth_manager()
am.init()
if app:
app.state.auth_manager = am

if app and (auth_manager_fastapi_app := am.get_fastapi_app()):
app.mount("/auth", auth_manager_fastapi_app)

return am


def get_auth_manager() -> BaseAuthManager:
"""Return the auth manager, provided it's been initialized before."""
global auth_manager

if auth_manager is None:
raise RuntimeError(
"Auth Manager has not been initialized yet. "
"The `init_auth_manager` method needs to be called first."
)
return auth_manager


def init_plugins(app: FastAPI) -> None:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,10 @@

from fastapi import HTTPException, status

from airflow.api_fastapi.app import get_auth_manager
from airflow.api_fastapi.auth.managers.simple.datamodels.login import LoginBody
from airflow.api_fastapi.auth.managers.simple.simple_auth_manager import SimpleAuthManager
from airflow.api_fastapi.auth.managers.simple.user import SimpleAuthManagerUser
from airflow.api_fastapi.common.auth_manager import AuthManagerDep
from airflow.configuration import conf


Expand All @@ -31,7 +31,9 @@ class SimpleAuthManagerLogin:

@staticmethod
def create_token(
body: LoginBody, expiration_time_in_seconds: int = conf.getint("api_auth", "jwt_expiration_time")
auth_manager: AuthManagerDep,
body: LoginBody,
expiration_time_in_seconds: int = conf.getint("api_auth", "jwt_expiration_time"),
) -> str:
"""
Authenticate user with given configuration.
Expand Down Expand Up @@ -70,9 +72,7 @@ def create_token(
role=found_users[0]["role"],
)

return get_auth_manager().generate_jwt(
user=user, expiration_time_in_seconds=expiration_time_in_seconds
)
return auth_manager.generate_jwt(user=user, expiration_time_in_seconds=expiration_time_in_seconds)

@staticmethod
def create_token_all_admins(
Expand All @@ -91,12 +91,11 @@ def create_token_all_admins(

@staticmethod
def _create_anonymous_admin_user(
auth_manager: AuthManagerDep,
expiration_time_in_seconds: int = conf.getint("api_auth", "jwt_expiration_time"),
) -> str:
user = SimpleAuthManagerUser(
username="Anonymous",
role="ADMIN",
)
return get_auth_manager().generate_jwt(
user=user, expiration_time_in_seconds=expiration_time_in_seconds
)
return auth_manager.generate_jwt(user=user, expiration_time_in_seconds=expiration_time_in_seconds)
93 changes: 93 additions & 0 deletions airflow-core/src/airflow/api_fastapi/common/auth_manager.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
# 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

from typing import TYPE_CHECKING, Annotated

from fastapi import Depends, Request

from airflow.api_fastapi.auth.managers.base_auth_manager import BaseAuthManager
from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException

if TYPE_CHECKING:
from fastapi import FastAPI

auth_manager: BaseAuthManager | None = None


def get_auth_manager_cls() -> type[BaseAuthManager]:
"""
Return just the auth manager class without initializing it.

Useful to save execution time if only static methods need to be called.
"""
auth_manager_cls = conf.getimport(section="core", key="auth_manager")

if not auth_manager_cls:
raise AirflowConfigException(
"No auth manager defined in the config. Please specify one using section/key [core/auth_manager]."
)

return auth_manager_cls


def create_auth_manager() -> BaseAuthManager:
"""Create the auth manager."""
global auth_manager

auth_manager_cls = get_auth_manager_cls()
auth_manager = auth_manager_cls()
return auth_manager


def get_auth_manager() -> BaseAuthManager:
"""Return the auth manager, provided it's been initialized before."""
global auth_manager

if auth_manager is None:
raise RuntimeError(
"Auth Manager has not been initialized yet. "
"The `init_auth_manager` method needs to be called first."
)
return auth_manager


def init_auth_manager(app: FastAPI | None = None) -> BaseAuthManager:
"""Initialize the auth manager for the FastAPI app."""
am = create_auth_manager()
am.init()
if app:
app.state.auth_manager = am

if app and (auth_manager_fastapi_app := am.get_fastapi_app()):
app.mount("/auth", auth_manager_fastapi_app)

return am


def dag_auth_manager_from_app(request: Request) -> BaseAuthManager:
"""
FastAPI dependency resolver that returns the shared auth manager instance from app.state.

This ensures that all API routes using auth manager via dependency injection receive the same
singleton instance that was initialized at app startup.
"""
return request.app.state.auth_manager


AuthManagerDep = Annotated[BaseAuthManager, Depends(dag_auth_manager_from_app)]
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,11 @@
from fastapi import Depends, HTTPException, status
from sqlalchemy import select

from airflow.api_fastapi.app import get_auth_manager
from airflow.api_fastapi.auth.managers.models.batch_apis import IsAuthorizedDagRequest
from airflow.api_fastapi.auth.managers.models.resource_details import (
DagDetails,
)
from airflow.api_fastapi.common.auth_manager import AuthManagerDep
from airflow.api_fastapi.common.db.common import (
SessionDep,
paginated_select,
Expand Down Expand Up @@ -67,6 +67,7 @@ def get_import_error(
import_error_id: int,
session: SessionDep,
user: GetUserDep,
auth_manager: AuthManagerDep,
) -> ImportErrorResponse:
"""Get an import error."""
error = session.scalar(select(ParseImportError).where(ParseImportError.id == import_error_id))
Expand All @@ -77,7 +78,6 @@ def get_import_error(
)
session.expunge(error)

auth_manager = get_auth_manager()
can_read_all_dags = auth_manager.is_authorized_dag(method="GET", user=user)
if can_read_all_dags:
# Early return if the user has access to all DAGs
Expand Down Expand Up @@ -128,6 +128,7 @@ def get_import_errors(
],
session: SessionDep,
user: GetUserDep,
auth_manager: AuthManagerDep,
) -> ImportErrorCollectionResponse:
"""Get all import errors."""
import_errors_select, total_entries = paginated_select(
Expand All @@ -138,7 +139,6 @@ def get_import_errors(
session=session,
)

auth_manager = get_auth_manager()
can_read_all_dags = auth_manager.is_authorized_dag(method="GET", user=user)
if can_read_all_dags:
# Early return if the user has access to all DAGs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

from __future__ import annotations

from airflow.api_fastapi.app import get_auth_manager
from airflow.api_fastapi.common.auth_manager import AuthManagerDep
from airflow.api_fastapi.common.router import AirflowRouter
from airflow.api_fastapi.core_api.datamodels.ui.auth import (
MenuItemCollectionResponse,
Expand All @@ -29,10 +29,11 @@

@auth_router.get("/auth/menus")
def get_auth_menus(
auth_manager: AuthManagerDep,
user: GetUserDep,
) -> MenuItemCollectionResponse:
authorized_menu_items = get_auth_manager().get_authorized_menu_items(user=user)
extra_menu_items = get_auth_manager().get_extra_menu_items(user=user)
authorized_menu_items = auth_manager.get_authorized_menu_items(user=user)
extra_menu_items = auth_manager.get_extra_menu_items(user=user)

return MenuItemCollectionResponse(
authorized_menu_items=authorized_menu_items,
Expand Down
Loading
Loading