Skip to content

Commit

Permalink
Implement login and logout in AWS auth manager (#35488)
Browse files Browse the repository at this point in the history
  • Loading branch information
vincbeck authored Nov 21, 2023
1 parent 1fae1a5 commit 379b7c0
Show file tree
Hide file tree
Showing 38 changed files with 1,045 additions and 114 deletions.
6 changes: 3 additions & 3 deletions CONTRIBUTING.rst
Original file line number Diff line number Diff line change
Expand Up @@ -679,9 +679,9 @@ github_enterprise, google, google_auth, grpc, hashicorp, hdfs, hive, http, imap,
jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp,
microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openai, openfaas, openlineage, opensearch,
opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, plexus,
postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, segment, sendgrid, sentry, sftp,
singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino,
vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry,
sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram,
trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
.. END EXTRAS HERE
Provider packages
Expand Down
4 changes: 2 additions & 2 deletions Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ function get_dev_apt_deps() {
DEV_APT_DEPS="apt-transport-https apt-utils build-essential ca-certificates dirmngr \
freetds-bin freetds-dev git gosu graphviz graphviz-dev krb5-user ldap-utils libffi-dev libgeos-dev \
libkrb5-dev libldap2-dev libleveldb1d libleveldb-dev libsasl2-2 libsasl2-dev libsasl2-modules \
libssl-dev locales lsb-release openssh-client pkgconf sasl2-bin \
libssl-dev libxmlsec1 libxmlsec1-dev locales lsb-release openssh-client pkgconf sasl2-bin \
software-properties-common sqlite3 sudo unixodbc unixodbc-dev"
export DEV_APT_DEPS
fi
Expand All @@ -123,7 +123,7 @@ function get_runtime_apt_deps() {
if [[ "${RUNTIME_APT_DEPS=}" == "" ]]; then
RUNTIME_APT_DEPS="apt-transport-https apt-utils ca-certificates \
curl dumb-init freetds-bin gosu krb5-user libgeos-dev \
ldap-utils libsasl2-2 libsasl2-modules locales ${debian_version_apt_deps} \
ldap-utils libsasl2-2 libsasl2-modules libxmlsec1 locales ${debian_version_apt_deps} \
lsb-release openssh-client python3-selinux rsync sasl2-bin sqlite3 sudo unixodbc"
export RUNTIME_APT_DEPS
fi
Expand Down
4 changes: 2 additions & 2 deletions Dockerfile.ci
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ function get_dev_apt_deps() {
DEV_APT_DEPS="apt-transport-https apt-utils build-essential ca-certificates dirmngr \
freetds-bin freetds-dev git gosu graphviz graphviz-dev krb5-user ldap-utils libffi-dev libgeos-dev \
libkrb5-dev libldap2-dev libleveldb1d libleveldb-dev libsasl2-2 libsasl2-dev libsasl2-modules \
libssl-dev locales lsb-release openssh-client pkgconf sasl2-bin \
libssl-dev libxmlsec1 libxmlsec1-dev locales lsb-release openssh-client pkgconf sasl2-bin \
software-properties-common sqlite3 sudo unixodbc unixodbc-dev"
export DEV_APT_DEPS
fi
Expand All @@ -83,7 +83,7 @@ function get_runtime_apt_deps() {
if [[ "${RUNTIME_APT_DEPS=}" == "" ]]; then
RUNTIME_APT_DEPS="apt-transport-https apt-utils ca-certificates \
curl dumb-init freetds-bin gosu krb5-user libgeos-dev \
ldap-utils libsasl2-2 libsasl2-modules locales ${debian_version_apt_deps} \
ldap-utils libsasl2-2 libsasl2-modules libxmlsec1 locales ${debian_version_apt_deps} \
lsb-release openssh-client python3-selinux rsync sasl2-bin sqlite3 sudo unixodbc"
export RUNTIME_APT_DEPS
fi
Expand Down
6 changes: 3 additions & 3 deletions INSTALL
Original file line number Diff line number Diff line change
Expand Up @@ -105,9 +105,9 @@ github_enterprise, google, google_auth, grpc, hashicorp, hdfs, hive, http, imap,
jenkins, kerberos, kubernetes, ldap, leveldb, microsoft.azure, microsoft.mssql, microsoft.psrp,
microsoft.winrm, mongo, mssql, mysql, neo4j, odbc, openai, openfaas, openlineage, opensearch,
opsgenie, oracle, otel, pagerduty, pandas, papermill, password, pgvector, pinecone, pinot, plexus,
postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, segment, sendgrid, sentry, sftp,
singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram, trino,
vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
postgres, presto, rabbitmq, redis, s3, s3fs, salesforce, samba, saml, segment, sendgrid, sentry,
sftp, singularity, slack, smtp, snowflake, spark, sqlite, ssh, statsd, tableau, tabular, telegram,
trino, vertica, virtualenv, weaviate, webhdfs, winrm, yandex, zendesk
# END EXTRAS HERE

# For installing Airflow in development environments - see CONTRIBUTING.rst
Expand Down
32 changes: 22 additions & 10 deletions airflow/auth/managers/base_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@

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

from airflow.auth.managers.models.base_user import BaseUser
Expand Down Expand Up @@ -64,11 +63,12 @@ class BaseAuthManager(LoggingMixin):
Class to derive in order to implement concrete auth managers.
Auth managers are responsible for any user management related operation such as login, logout, authz, ...
:param appbuilder: the flask app builder
"""

def __init__(self, app: Flask, appbuilder: AirflowAppBuilder) -> None:
def __init__(self, appbuilder: AirflowAppBuilder) -> None:
super().__init__()
self.app = app
self.appbuilder = appbuilder

@staticmethod
Expand All @@ -83,21 +83,29 @@ 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."""
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()

@abstractmethod
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) -> BaseUser:
def get_user(self) -> BaseUser | None:
"""Return the user associated to the user in session."""

@abstractmethod
def get_user_id(self) -> str:
"""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.")
return str(user.get_id())

def init(self) -> None:
"""
Expand Down Expand Up @@ -384,17 +392,21 @@ def get_url_login(self, **kwargs) -> str:
def get_url_logout(self) -> str:
"""Return the logout page url."""

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

@cached_property
def security_manager(self) -> AirflowSecurityManagerV2:
"""
Return the security manager.
By default, Airflow comes with the default security manager
airflow.www.security_manager.AirflowSecurityManagerV2. The auth manager might need to extend this
``airflow.www.security_manager.AirflowSecurityManagerV2``. The auth manager might need to extend this
default security manager for its own purposes.
By default, return the default AirflowSecurityManagerV2.
Expand Down
16 changes: 1 addition & 15 deletions airflow/auth/managers/fab/fab_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -170,26 +170,12 @@ def get_user_display_name(self) -> str:
last_name = user.last_name.strip() if isinstance(user.last_name, str) else ""
return f"{first_name} {last_name}".strip()

def get_user_name(self) -> str:
"""
Return the username associated to the user in session.
For backward compatibility reasons, the username in FAB auth manager can be any of username,
email, or the database user ID.
"""
user = self.get_user()
return user.username or user.email or self.get_user_id()

def get_user(self) -> User:
"""Return the user associated to the user in session."""
from flask_login import current_user

return current_user

def get_user_id(self) -> str:
"""Return the user ID associated to the user in session."""
return str(self.get_user().get_id())

def init(self) -> None:
"""Run operations when Airflow is initializing."""
self._sync_appbuilder_roles()
Expand Down Expand Up @@ -351,7 +337,7 @@ def security_manager(self) -> FabAirflowSecurityManagerOverride:
from airflow.auth.managers.fab.security_manager.override import FabAirflowSecurityManagerOverride
from airflow.www.security import AirflowSecurityManager

sm_from_config = self.app.config.get("SECURITY_MANAGER_CLASS")
sm_from_config = self.appbuilder.get_app.config.get("SECURITY_MANAGER_CLASS")
if sm_from_config:
if not issubclass(sm_from_config, AirflowSecurityManager):
raise Exception(
Expand Down
3 changes: 3 additions & 0 deletions airflow/auth/managers/fab/models/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,9 @@ def perms(self):
def get_id(self):
return self.id

def get_name(self) -> str:
return self.username or self.email or self.user_id

def get_full_name(self):
return f"{self.first_name} {self.last_name}"

Expand Down
7 changes: 5 additions & 2 deletions airflow/auth/managers/models/base_user.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,13 @@ class BaseUser:
"""User model interface."""

@property
@abstractmethod
def is_active(self) -> bool:
...
return True

@abstractmethod
def get_id(self) -> str:
...

@abstractmethod
def get_name(self) -> str:
...
16 changes: 16 additions & 0 deletions airflow/providers/amazon/aws/auth_manager/__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.
143 changes: 143 additions & 0 deletions airflow/providers/amazon/aws/auth_manager/aws_auth_manager.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
# 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 functools import cached_property
from typing import TYPE_CHECKING

from flask import session, url_for

from airflow.configuration import conf
from airflow.exceptions import AirflowOptionalProviderFeatureException
from airflow.providers.amazon.aws.auth_manager.constants import (
CONF_ENABLE_KEY,
CONF_SECTION_NAME,
)
from airflow.providers.amazon.aws.auth_manager.security_manager.aws_security_manager_override import (
AwsSecurityManagerOverride,
)

try:
from airflow.auth.managers.base_auth_manager import BaseAuthManager, ResourceMethod
except ImportError:
raise AirflowOptionalProviderFeatureException(
"Failed to import BaseUser. This feature is only available in Airflow versions >= 2.8.0"
)

if TYPE_CHECKING:
from airflow.auth.managers.models.base_user import BaseUser
from airflow.auth.managers.models.resource_details import (
AccessView,
ConfigurationDetails,
ConnectionDetails,
DagAccessEntity,
DagDetails,
DatasetDetails,
PoolDetails,
VariableDetails,
)
from airflow.providers.amazon.aws.auth_manager.user import AwsAuthManagerUser
from airflow.www.extensions.init_appbuilder import AirflowAppBuilder


class AwsAuthManager(BaseAuthManager):
"""
AWS auth manager.
Leverages AWS services such as Amazon Identity Center and Amazon Verified Permissions to perform
authentication and authorization in Airflow.
:param appbuilder: the flask app builder
"""

def __init__(self, appbuilder: AirflowAppBuilder) -> None:
super().__init__(appbuilder)
enable = conf.getboolean(CONF_SECTION_NAME, CONF_ENABLE_KEY)
if not enable:
raise NotImplementedError(
"The AWS auth manager is currently being built. It is not finalized. It is not intended to be used yet."
)

def get_user(self) -> AwsAuthManagerUser | None:
return session["aws_user"] if self.is_logged_in() else None

def is_logged_in(self) -> bool:
return "aws_user" in session

def is_authorized_configuration(
self,
*,
method: ResourceMethod,
details: ConfigurationDetails | None = None,
user: BaseUser | None = None,
) -> bool:
return self.is_logged_in()

def is_authorized_cluster_activity(self, *, method: ResourceMethod, user: BaseUser | None = None) -> bool:
return self.is_logged_in()

def is_authorized_connection(
self,
*,
method: ResourceMethod,
details: ConnectionDetails | None = None,
user: BaseUser | None = None,
) -> bool:
return self.is_logged_in()

def is_authorized_dag(
self,
*,
method: ResourceMethod,
access_entity: DagAccessEntity | None = None,
details: DagDetails | None = None,
user: BaseUser | None = None,
) -> bool:
return self.is_logged_in()

def is_authorized_dataset(
self, *, method: ResourceMethod, details: DatasetDetails | None = None, user: BaseUser | None = None
) -> bool:
return self.is_logged_in()

def is_authorized_pool(
self, *, method: ResourceMethod, details: PoolDetails | None = None, user: BaseUser | None = None
) -> bool:
return self.is_logged_in()

def is_authorized_variable(
self, *, method: ResourceMethod, details: VariableDetails | None = None, user: BaseUser | None = None
) -> bool:
return self.is_logged_in()

def is_authorized_view(
self,
*,
access_view: AccessView,
user: BaseUser | None = None,
) -> bool:
return self.is_logged_in()

def get_url_login(self, **kwargs) -> str:
return url_for("AwsAuthManagerAuthenticationViews.login")

def get_url_logout(self) -> str:
return url_for("AwsAuthManagerAuthenticationViews.logout")

@cached_property
def security_manager(self) -> AwsSecurityManagerOverride:
return AwsSecurityManagerOverride(self.appbuilder)
23 changes: 23 additions & 0 deletions airflow/providers/amazon/aws/auth_manager/constants.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
# 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.

# Configuration keys
from __future__ import annotations

CONF_SECTION_NAME = "aws_auth_manager"
CONF_SAML_METADATA_URL_KEY = "saml_metadata_url"
CONF_ENABLE_KEY = "enable"
Loading

0 comments on commit 379b7c0

Please sign in to comment.