Skip to content

Commit

Permalink
Move Rest API auth related methods to FAB auth manager (#34924)
Browse files Browse the repository at this point in the history
  • Loading branch information
vincbeck authored Oct 20, 2023
1 parent eb76276 commit 48c17fb
Show file tree
Hide file tree
Showing 11 changed files with 654 additions and 482 deletions.
56 changes: 19 additions & 37 deletions airflow/api/auth/backend/basic_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,56 +14,38 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""Basic authentication backend."""
from __future__ import annotations
"""
This module is deprecated.
from functools import wraps
from typing import TYPE_CHECKING, Any, Callable, TypeVar, cast
Please use :mod:`airflow.auth.managers.fab.api.auth.backend.basic_auth` instead.
"""
from __future__ import annotations

from flask import Response, request
from flask_appbuilder.const import AUTH_LDAP
from flask_login import login_user
import warnings
from typing import TYPE_CHECKING, Any, Callable

from airflow.utils.airflow_flask_app import get_airflow_app
import airflow.auth.managers.fab.api.auth.backend.basic_auth as fab_basic_auth
from airflow.exceptions import RemovedInAirflow3Warning

if TYPE_CHECKING:
from airflow.auth.managers.fab.models import User

CLIENT_AUTH: tuple[str, str] | Any | None = None


def init_app(_):
"""Initialize authentication backend."""
warnings.warn(
"This module is deprecated. Please use `airflow.auth.managers.fab.api.auth.backend.basic_auth` instead.",
RemovedInAirflow3Warning,
stacklevel=2,
)


T = TypeVar("T", bound=Callable)
def init_app(_):
fab_basic_auth.init_app(_)


def auth_current_user() -> User | None:
"""Authenticate and set current user if Authorization header exists."""
auth = request.authorization
if auth is None or not auth.username or not auth.password:
return None

ab_security_manager = get_airflow_app().appbuilder.sm
user = None
if ab_security_manager.auth_type == AUTH_LDAP:
user = ab_security_manager.auth_user_ldap(auth.username, auth.password)
if user is None:
user = ab_security_manager.auth_user_db(auth.username, auth.password)
if user is not None:
login_user(user, remember=False)
return user


def requires_authentication(function: T):
"""Decorate functions that require authentication."""
return fab_basic_auth.auth_current_user()

@wraps(function)
def decorated(*args, **kwargs):
if auth_current_user() is not None:
return function(*args, **kwargs)
else:
return Response("Unauthorized", 401, {"WWW-Authenticate": "Basic"})

return cast(T, decorated)
def requires_authentication(function: Callable):
return fab_basic_auth.requires_authentication(function)
20 changes: 17 additions & 3 deletions airflow/api/auth/backend/kerberos_auth.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,9 @@
# under the License.
from __future__ import annotations

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.utils.airflow_flask_app import get_airflow_app

#
Expand Down Expand Up @@ -46,7 +49,7 @@
import logging
import os
from functools import wraps
from typing import Any, Callable, TypeVar, cast
from typing import TYPE_CHECKING, Any, Callable, TypeVar, cast

import kerberos
from flask import Response, _request_ctx_stack as stack, g, make_response, request # type: ignore
Expand All @@ -55,6 +58,9 @@
from airflow.configuration import conf
from airflow.utils.net import getfqdn

if TYPE_CHECKING:
from airflow.auth.managers.models.base_user import BaseUser

log = logging.getLogger(__name__)


Expand Down Expand Up @@ -129,8 +135,16 @@ def _gssapi_authenticate(token):
T = TypeVar("T", bound=Callable)


def requires_authentication(function: T):
def requires_authentication(function: T, find_user: Callable[[str], BaseUser] | None = None):
"""Decorate functions that require authentication with Kerberos."""
if not find_user:
warnings.warn(
"This module is deprecated. Please use "
"`airflow.auth.managers.fab.api.auth.backend.kerberos_auth` instead.",
RemovedInAirflow3Warning,
stacklevel=2,
)
find_user = get_airflow_app().appbuilder.sm.find_user

@wraps(function)
def decorated(*args, **kwargs):
Expand All @@ -140,7 +154,7 @@ def decorated(*args, **kwargs):
token = "".join(header.split()[1:])
return_code = _gssapi_authenticate(token)
if return_code == kerberos.AUTH_GSS_COMPLETE:
g.user = get_airflow_app().appbuilder.sm.find_user(username=ctx.kerberos_user)
g.user = find_user(ctx.kerberos_user)
response = function(*args, **kwargs)
response = make_response(response)
if ctx.kerberos_token is not None:
Expand Down
17 changes: 17 additions & 0 deletions airflow/auth/managers/fab/api/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
17 changes: 17 additions & 0 deletions airflow/auth/managers/fab/api/auth/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
17 changes: 17 additions & 0 deletions airflow/auth/managers/fab/api/auth/backend/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
68 changes: 68 additions & 0 deletions airflow/auth/managers/fab/api/auth/backend/basic_auth.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
# 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.
"""Basic authentication backend."""
from __future__ import annotations

from functools import wraps
from typing import TYPE_CHECKING, Any, Callable, TypeVar, cast

from flask import Response, request
from flask_appbuilder.const import AUTH_LDAP
from flask_login import login_user

from airflow.utils.airflow_flask_app import get_airflow_app

if TYPE_CHECKING:
from airflow.auth.managers.fab.models import User

CLIENT_AUTH: tuple[str, str] | Any | None = None

T = TypeVar("T", bound=Callable)


def init_app(_):
"""Initialize authentication backend."""


def auth_current_user() -> User | None:
"""Authenticate and set current user if Authorization header exists."""
auth = request.authorization
if auth is None or not auth.username or not auth.password:
return None

ab_security_manager = get_airflow_app().appbuilder.sm
user = None
if ab_security_manager.auth_type == AUTH_LDAP:
user = ab_security_manager.auth_user_ldap(auth.username, auth.password)
if user is None:
user = ab_security_manager.auth_user_db(auth.username, auth.password)
if user is not None:
login_user(user, remember=False)
return user


def requires_authentication(function: T):
"""Decorate functions that require authentication."""

@wraps(function)
def decorated(*args, **kwargs):
if auth_current_user() is not None:
return function(*args, **kwargs)
else:
return Response("Unauthorized", 401, {"WWW-Authenticate": "Basic"})

return cast(T, decorated)
39 changes: 39 additions & 0 deletions airflow/auth/managers/fab/api/auth/backend/kerberos_auth.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
#
# 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 logging
from functools import partial
from typing import Any

from requests_kerberos import HTTPKerberosAuth

from airflow.api.auth.backend.kerberos_auth import (
init_app as base_init_app,
requires_authentication as base_requires_authentication,
)
from airflow.utils.airflow_flask_app import get_airflow_app

log = logging.getLogger(__name__)

CLIENT_AUTH: tuple[str, str] | Any | None = HTTPKerberosAuth(service="airflow")

init_app = base_init_app
requires_authentication = partial(
base_requires_authentication, find_user=get_airflow_app().appbuilder.sm.find_user
)
Loading

0 comments on commit 48c17fb

Please sign in to comment.