You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/04/20 16:48:45 UTC

[GitHub] [airflow] jedcunningham commented on a change in pull request #15042: Add REST API auth flow for the new UI

jedcunningham commented on a change in pull request #15042:
URL: https://github.com/apache/airflow/pull/15042#discussion_r616824934



##########
File path: airflow/api_connexion/endpoints/auth_endpoint.py
##########
@@ -0,0 +1,158 @@
+# 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.
+import logging
+
+from flask import current_app, jsonify, request, session as c_session
+from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_OID, AUTH_REMOTE_USER
+from flask_jwt_extended import (
+    create_access_token,
+    decode_token,
+    get_jwt_identity,
+    get_raw_jwt,
+    jwt_refresh_token_required,
+    jwt_required,
+)
+from flask_login import login_user
+from marshmallow import ValidationError
+
+from airflow.api_connexion.exceptions import BadRequest, Unauthenticated
+from airflow.api_connexion.schemas.auth_schema import info_schema, login_form_schema, token_schema
+from airflow.models.auth import TokenBlockList
+
+log = logging.getLogger(__name__)
+
+
+def get_auth_info():
+    """Get site authentication info"""
+    security_manager = current_app.appbuilder.sm
+    config = current_app.config
+    auth_type = security_manager.auth_type
+    type_mapping = {
+        AUTH_DB: "auth_db",
+        AUTH_LDAP: "auth_ldap",
+        AUTH_OID: "auth_oid",
+        AUTH_OAUTH: "auth_oauth",
+        AUTH_REMOTE_USER: "auth_remote_user",
+    }
+    oauth_providers = config.get("OAUTH_PROVIDERS", None)
+    openid_providers = config.get("OPENID_PROVIDERS", None)
+    return info_schema.dump(
+        {
+            "auth_type": type_mapping[auth_type],
+            "oauth_providers": oauth_providers,
+            "openid_providers": openid_providers,
+        }
+    )
+
+
+def auth_login():
+    """Handle DB login"""
+    body = request.json
+    try:
+        data = login_form_schema.load(body)
+    except ValidationError as err:
+        raise Unauthenticated(detail=str(err.messages))

Review comment:
       ```suggestion
           raise BadRequest(detail=str(err.messages))
   ```

##########
File path: airflow/api_connexion/endpoints/auth_endpoint.py
##########
@@ -0,0 +1,158 @@
+# 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.
+import logging
+
+from flask import current_app, jsonify, request, session as c_session
+from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_OID, AUTH_REMOTE_USER
+from flask_jwt_extended import (
+    create_access_token,
+    decode_token,
+    get_jwt_identity,
+    get_raw_jwt,
+    jwt_refresh_token_required,
+    jwt_required,
+)
+from flask_login import login_user
+from marshmallow import ValidationError
+
+from airflow.api_connexion.exceptions import BadRequest, Unauthenticated
+from airflow.api_connexion.schemas.auth_schema import info_schema, login_form_schema, token_schema
+from airflow.models.auth import TokenBlockList
+
+log = logging.getLogger(__name__)
+
+
+def get_auth_info():
+    """Get site authentication info"""
+    security_manager = current_app.appbuilder.sm
+    config = current_app.config
+    auth_type = security_manager.auth_type
+    type_mapping = {
+        AUTH_DB: "auth_db",
+        AUTH_LDAP: "auth_ldap",
+        AUTH_OID: "auth_oid",
+        AUTH_OAUTH: "auth_oauth",
+        AUTH_REMOTE_USER: "auth_remote_user",
+    }
+    oauth_providers = config.get("OAUTH_PROVIDERS", None)
+    openid_providers = config.get("OPENID_PROVIDERS", None)
+    return info_schema.dump(
+        {
+            "auth_type": type_mapping[auth_type],
+            "oauth_providers": oauth_providers,
+            "openid_providers": openid_providers,
+        }
+    )
+
+
+def auth_login():
+    """Handle DB login"""
+    body = request.json
+    try:
+        data = login_form_schema.load(body)
+    except ValidationError as err:
+        raise Unauthenticated(detail=str(err.messages))
+    security_manager = current_app.appbuilder.sm
+    user = security_manager.login_with_user_pass(data['username'], data['password'])
+    if not user:
+        raise Unauthenticated(detail="Invalid login")
+    login_user(user, remember=False)
+    return security_manager.create_tokens_and_dump(user)
+
+
+def auth_oauthlogin(provider, register=None, redirect_url=None):
+    """Returns OAUTH authorization url"""
+    appbuilder = current_app.appbuilder
+    if register:
+        c_session["register"] = True
+    return appbuilder.sm.oauth_authorization_url(appbuilder.app, provider, redirect_url)
+
+
+def authorize_oauth(provider, state):
+    """Callback to authorize Oauth."""
+    appbuilder = current_app.appbuilder
+    user = appbuilder.sm.oauth_login_user(appbuilder.app, provider, state)
+    return appbuilder.sm.create_tokens_and_dump(user)
+
+
+def auth_remoteuser():
+    """Handle remote user auth"""
+    appbuilder = current_app.appbuilder
+    username = request.environ.get("REMOTE_USER")
+    if username:
+        user = appbuilder.sm.login_remote_user(username)
+    else:
+        raise Unauthenticated(detail="Invalid login")

Review comment:
       nit:
   ```suggestion
       if not username:
           raise Unauthenticated(detail="Invalid login")
       user = appbuilder.sm.login_remote_user(username)
   ```

##########
File path: airflow/api_connexion/endpoints/auth_endpoint.py
##########
@@ -0,0 +1,158 @@
+# 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.
+import logging
+
+from flask import current_app, jsonify, request, session as c_session
+from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_OID, AUTH_REMOTE_USER
+from flask_jwt_extended import (
+    create_access_token,
+    decode_token,
+    get_jwt_identity,
+    get_raw_jwt,
+    jwt_refresh_token_required,
+    jwt_required,
+)
+from flask_login import login_user
+from marshmallow import ValidationError
+
+from airflow.api_connexion.exceptions import BadRequest, Unauthenticated
+from airflow.api_connexion.schemas.auth_schema import info_schema, login_form_schema, token_schema
+from airflow.models.auth import TokenBlockList
+
+log = logging.getLogger(__name__)
+
+
+def get_auth_info():
+    """Get site authentication info"""
+    security_manager = current_app.appbuilder.sm
+    config = current_app.config
+    auth_type = security_manager.auth_type
+    type_mapping = {
+        AUTH_DB: "auth_db",
+        AUTH_LDAP: "auth_ldap",
+        AUTH_OID: "auth_oid",
+        AUTH_OAUTH: "auth_oauth",
+        AUTH_REMOTE_USER: "auth_remote_user",
+    }
+    oauth_providers = config.get("OAUTH_PROVIDERS", None)
+    openid_providers = config.get("OPENID_PROVIDERS", None)
+    return info_schema.dump(
+        {
+            "auth_type": type_mapping[auth_type],
+            "oauth_providers": oauth_providers,
+            "openid_providers": openid_providers,
+        }
+    )
+
+
+def auth_login():
+    """Handle DB login"""
+    body = request.json
+    try:
+        data = login_form_schema.load(body)
+    except ValidationError as err:
+        raise Unauthenticated(detail=str(err.messages))
+    security_manager = current_app.appbuilder.sm
+    user = security_manager.login_with_user_pass(data['username'], data['password'])
+    if not user:
+        raise Unauthenticated(detail="Invalid login")
+    login_user(user, remember=False)
+    return security_manager.create_tokens_and_dump(user)
+
+
+def auth_oauthlogin(provider, register=None, redirect_url=None):
+    """Returns OAUTH authorization url"""
+    appbuilder = current_app.appbuilder
+    if register:
+        c_session["register"] = True
+    return appbuilder.sm.oauth_authorization_url(appbuilder.app, provider, redirect_url)
+
+
+def authorize_oauth(provider, state):
+    """Callback to authorize Oauth."""
+    appbuilder = current_app.appbuilder
+    user = appbuilder.sm.oauth_login_user(appbuilder.app, provider, state)
+    return appbuilder.sm.create_tokens_and_dump(user)
+
+
+def auth_remoteuser():
+    """Handle remote user auth"""
+    appbuilder = current_app.appbuilder
+    username = request.environ.get("REMOTE_USER")
+    if username:
+        user = appbuilder.sm.login_remote_user(username)
+    else:
+        raise Unauthenticated(detail="Invalid login")
+    return appbuilder.sm.create_tokens_and_dump(user)
+
+
+@jwt_refresh_token_required
+def refresh_token():
+    """Refresh token"""
+    user = get_jwt_identity()
+    access_token = create_access_token(identity=user)
+    ret = {'access_token': access_token}
+    return jsonify(ret), 200
+
+
+@jwt_required
+def revoke_token():
+    """
+    An endpoint for revoking both access and refresh token.
+
+    This is intended for a case where a logged in user want to revoke
+    another user's tokens
+    """
+    resp = jsonify({"revoked": True})

Review comment:
       Just return this at the bottom instead of shoving it in a variable.

##########
File path: airflow/api_connexion/endpoints/auth_endpoint.py
##########
@@ -0,0 +1,158 @@
+# 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.
+import logging
+
+from flask import current_app, jsonify, request, session as c_session
+from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_OID, AUTH_REMOTE_USER
+from flask_jwt_extended import (
+    create_access_token,
+    decode_token,
+    get_jwt_identity,
+    get_raw_jwt,
+    jwt_refresh_token_required,
+    jwt_required,
+)
+from flask_login import login_user
+from marshmallow import ValidationError
+
+from airflow.api_connexion.exceptions import BadRequest, Unauthenticated
+from airflow.api_connexion.schemas.auth_schema import info_schema, login_form_schema, token_schema
+from airflow.models.auth import TokenBlockList
+
+log = logging.getLogger(__name__)
+
+
+def get_auth_info():
+    """Get site authentication info"""
+    security_manager = current_app.appbuilder.sm
+    config = current_app.config
+    auth_type = security_manager.auth_type
+    type_mapping = {
+        AUTH_DB: "auth_db",
+        AUTH_LDAP: "auth_ldap",
+        AUTH_OID: "auth_oid",
+        AUTH_OAUTH: "auth_oauth",
+        AUTH_REMOTE_USER: "auth_remote_user",
+    }
+    oauth_providers = config.get("OAUTH_PROVIDERS", None)
+    openid_providers = config.get("OPENID_PROVIDERS", None)
+    return info_schema.dump(
+        {
+            "auth_type": type_mapping[auth_type],
+            "oauth_providers": oauth_providers,
+            "openid_providers": openid_providers,
+        }
+    )
+
+
+def auth_login():
+    """Handle DB login"""
+    body = request.json
+    try:
+        data = login_form_schema.load(body)
+    except ValidationError as err:
+        raise Unauthenticated(detail=str(err.messages))
+    security_manager = current_app.appbuilder.sm
+    user = security_manager.login_with_user_pass(data['username'], data['password'])
+    if not user:
+        raise Unauthenticated(detail="Invalid login")
+    login_user(user, remember=False)
+    return security_manager.create_tokens_and_dump(user)
+
+
+def auth_oauthlogin(provider, register=None, redirect_url=None):
+    """Returns OAUTH authorization url"""
+    appbuilder = current_app.appbuilder
+    if register:
+        c_session["register"] = True
+    return appbuilder.sm.oauth_authorization_url(appbuilder.app, provider, redirect_url)
+
+
+def authorize_oauth(provider, state):
+    """Callback to authorize Oauth."""
+    appbuilder = current_app.appbuilder
+    user = appbuilder.sm.oauth_login_user(appbuilder.app, provider, state)
+    return appbuilder.sm.create_tokens_and_dump(user)
+
+
+def auth_remoteuser():
+    """Handle remote user auth"""
+    appbuilder = current_app.appbuilder
+    username = request.environ.get("REMOTE_USER")
+    if username:
+        user = appbuilder.sm.login_remote_user(username)
+    else:
+        raise Unauthenticated(detail="Invalid login")
+    return appbuilder.sm.create_tokens_and_dump(user)
+
+
+@jwt_refresh_token_required
+def refresh_token():
+    """Refresh token"""
+    user = get_jwt_identity()
+    access_token = create_access_token(identity=user)
+    ret = {'access_token': access_token}
+    return jsonify(ret), 200

Review comment:
       ```suggestion
       return jsonify({'access_token': access_token})
   ```

##########
File path: airflow/api_connexion/openapi/v1.yaml
##########
@@ -1572,6 +1572,196 @@ paths:
         '404':
           $ref: '#/components/responses/NotFound'
 
+  /auth-info:
+    get:
+      summary: Get site authentication information
+      x-openapi-router-controller: airflow.api_connexion.endpoints.auth_endpoint
+      operationId: get_auth_info
+      tags: [Authentication]
+      responses:
+        '200':
+          description: Success.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/AuthInfo'
+
+  /auth/login:
+    post:
+      summary: Login user
+      x-openapi-router-controller: airflow.api_connexion.endpoints.auth_endpoint
+      operationId: auth_login
+      tags: [Authentication]
+      requestBody:
+        required: true
+        content:
+          application/json:
+            schema:
+              $ref: '#/components/schemas/AuthDBLoginForm'
+      responses:
+        '200':
+          description: Success.
+          content:
+            application/json:
+              schema:
+                $ref: '#/components/schemas/JwtAuthAndUser'
+        '401':
+          $ref: '#/components/responses/Unauthenticated'

Review comment:
       This probably needs `400` too? Not sure about the rest of these.

##########
File path: airflow/api_connexion/security.py
##########
@@ -14,24 +14,38 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
 from functools import wraps
 from typing import Callable, Optional, Sequence, Tuple, TypeVar, cast
 
 from flask import Response, current_app
+from flask_jwt_extended import verify_jwt_in_request
 
 from airflow.api_connexion.exceptions import PermissionDenied, Unauthenticated
+from airflow.models import TokenBlockList
 
 T = TypeVar("T", bound=Callable)  # pylint: disable=invalid-name
 
 
+@current_app.appbuilder.sm.jwt_manager.token_in_blacklist_loader
+def check_if_token_in_blacklist(decrypted_token):
+    """Checks if there's a blocked token"""
+    jti = decrypted_token['jti']
+    return TokenBlockList.get_token(jti) is not None
+
+
 def check_authentication() -> None:
     """Checks that the request has valid authorization information."""
     response = current_app.api_auth.requires_authentication(Response)()
-    if response.status_code != 200:
+    if response.status_code == 200:
+        return
+    try:
+        verify_jwt_in_request()
+        return
+    except Exception:  # pylint: disable=broad-except

Review comment:
       This does seem overly broad? We wouldn't want to hide real 500's, right?

##########
File path: airflow/models/auth.py
##########
@@ -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.
+
+from pendulum import from_timestamp
+from sqlalchemy import Column, DateTime, Integer, String
+
+from airflow.models.base import Base
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.session import provide_session
+
+
+class TokenBlockList(Base, LoggingMixin):

Review comment:
       Should this be in `airflow/models/tokenblocklist.py` instead?

##########
File path: airflow/www/security.py
##########
@@ -728,3 +735,102 @@ def check_authorization(
                 return False
 
         return True
+
+    # TODO: Whether to create APISecurityManager and move api related code to it?
+    def is_user_logged_in(self):
+        """Raise if user already logged in"""
+        if g.user is not None and g.user.is_authenticated:
+            raise Unauthenticated(detail="Client already authenticated")  # For security
+
+    def login_with_user_pass(self, username, password):
+        """Convenience method for user login through the API"""
+        self.is_user_logged_in()
+        if self.auth_type not in (AUTH_DB, AUTH_LDAP):
+            raise Unauthenticated(detail="Authentication type do not match")

Review comment:
       Not sure this is the right response, should be a `BadRequest` right? The message also probably needs some adjustment imo.

##########
File path: airflow/www/security.py
##########
@@ -728,3 +735,102 @@ def check_authorization(
                 return False
 
         return True
+
+    # TODO: Whether to create APISecurityManager and move api related code to it?
+    def is_user_logged_in(self):
+        """Raise if user already logged in"""
+        if g.user is not None and g.user.is_authenticated:
+            raise Unauthenticated(detail="Client already authenticated")  # For security
+
+    def login_with_user_pass(self, username, password):
+        """Convenience method for user login through the API"""
+        self.is_user_logged_in()
+        if self.auth_type not in (AUTH_DB, AUTH_LDAP):
+            raise Unauthenticated(detail="Authentication type do not match")
+        user = None
+        if self.auth_type == AUTH_DB:
+            user = self.auth_user_db(username, password)
+        elif self.auth_type == AUTH_LDAP:
+            user = self.auth_user_ldap(username, password)
+        return user
+
+    def oauth_authorization_url(self, app, provider, redirect_url):
+        """Get authorization url for oauth"""
+        self.is_user_logged_in()
+        if self.auth_type != AUTH_OAUTH:
+            raise Unauthenticated(detail="Authentication type do not match")
+        state = jwt.encode(
+            request.args.to_dict(flat=False),
+            app.config["SECRET_KEY"],
+            algorithm="HS256",
+        )
+        auth_provider = self.oauth_remotes[provider]
+        try:
+
+            if provider == "twitter":
+                redirect_uri = redirect_url + f"&state={state}"
+                auth_data = auth_provider.create_authorization_url(redirect_uri=redirect_uri)
+                auth_provider.save_authorize_data(request, redirect_uri=redirect_uri, **auth_data)
+                return dict(auth_url=auth_data['url'])
+            else:
+                state = state.decode("ascii") if isinstance(state, bytes) else state
+                auth_data = auth_provider.create_authorization_url(
+                    redirect_uri=redirect_url,
+                    state=state,
+                )
+                auth_provider.save_authorize_data(request, redirect_uri=redirect_url, **auth_data)
+                return dict(auth_url=auth_data['url'])
+        except Exception as err:  # pylint: disable=broad-except

Review comment:
       This also seems broad and could mask true 500s.

##########
File path: airflow/www/security.py
##########
@@ -18,16 +18,23 @@
 #
 
 import warnings
+from datetime import timedelta
 from typing import Dict, Optional, Sequence, Set, Tuple
 
-from flask import current_app, g
+import jwt
+from flask import current_app, g, request
+from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_REMOTE_USER
 from flask_appbuilder.security.sqla import models as sqla_models
 from flask_appbuilder.security.sqla.manager import SecurityManager
 from flask_appbuilder.security.sqla.models import PermissionView, Role, User
+from flask_jwt_extended import JWTManager, create_access_token, create_refresh_token
+from flask_login import login_user
 from sqlalchemy import or_
 from sqlalchemy.orm import joinedload
 
 from airflow import models
+from airflow.api_connexion.exceptions import Unauthenticated
+from airflow.api_connexion.schemas.auth_schema import auth_schema

Review comment:
       I tend to agree. I think a slight refactor could keep the security manager cleaner and move the api_connextion specific stuff back to that side.
   
   e.g, take a look at how [auth_user_ldap](https://github.com/dpgaspar/Flask-AppBuilder/blob/27b15e59316e85e0fe62b8aa9978391ed4c729c9/flask_appbuilder/security/manager.py#L990) just returns `None` and lets the caller deal with the right response.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org