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
2 changes: 2 additions & 0 deletions airflow-core/src/airflow/api_fastapi/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
init_config,
init_error_handlers,
init_flask_plugins,
init_middlewares,
init_ui_plugins,
init_views,
)
Expand Down Expand Up @@ -99,6 +100,7 @@ def create_app(apps: str = "all") -> FastAPI:
init_ui_plugins(app)
init_views(app) # Core views need to be the last routes added - it has a catch all route
init_error_handlers(app)
init_middlewares(app)

init_config(app)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,12 +135,15 @@ def get_url_logout(self) -> str | None:
"""
return None

def get_url_refresh(self) -> str | None:
def refresh_user(self, *, user: T) -> T | None:
"""
Return the URL to refresh the authentication token.
Refresh the user if needed.

This is used to refresh the authentication token when it expires.
The default implementation returns None, which means that the auth manager does not support refresh token.
By default, does nothing. Some auth managers might need to refresh the user to, for instance,
refresh some tokens that are needed to communicate with a service/tool.

This method is called by every single request, it must be lightweight otherwise the overall API
server latency will increase.
"""
return None

Expand Down
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.
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.
from __future__ import annotations

from fastapi import Request
from starlette.middleware.base import BaseHTTPMiddleware

from airflow.api_fastapi.app import get_auth_manager
from airflow.api_fastapi.auth.managers.base_auth_manager import COOKIE_NAME_JWT_TOKEN
from airflow.api_fastapi.auth.managers.models.base_user import BaseUser
from airflow.api_fastapi.core_api.security import resolve_user_from_token
from airflow.configuration import conf


class JWTRefreshMiddleware(BaseHTTPMiddleware):
"""
Middleware to handle JWT token refresh.

This middleware:
1. Extracts JWT token from cookies and build the user from the token
2. Calls ``refresh_user`` method from auth manager with the user
3. If ``refresh_user`` returns a user, generate a JWT token based upon this user and send it in the
response as cookie
"""

async def dispatch(self, request: Request, call_next):
new_user = None
current_token = request.cookies.get(COOKIE_NAME_JWT_TOKEN)
if current_token:
new_user = await self._refresh_user(current_token)
if new_user:
request.state.user = new_user

response = await call_next(request)

if new_user:
# If we created a new user, serialize it and set it as a cookie
new_token = get_auth_manager().generate_jwt(new_user)
secure = bool(conf.get("api", "ssl_cert", fallback=""))
response.set_cookie(
COOKIE_NAME_JWT_TOKEN,
new_token,
httponly=True,
secure=secure,
samesite="lax",
)

return response

@staticmethod
async def _refresh_user(current_token: str) -> BaseUser | None:
user = await resolve_user_from_token(current_token)
return get_auth_manager().refresh_user(user=user)
6 changes: 6 additions & 0 deletions airflow-core/src/airflow/api_fastapi/core_api/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,12 @@ def init_error_handlers(app: FastAPI) -> None:
app.add_exception_handler(handler.exception_cls, handler.exception_handler)


def init_middlewares(app: FastAPI) -> None:
from airflow.api_fastapi.auth.middlewares.refresh_token import JWTRefreshMiddleware

app.add_middleware(JWTRefreshMiddleware)


def init_ui_plugins(app: FastAPI) -> None:
"""Initialize UI plugins."""
from airflow import plugins_manager
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8478,40 +8478,6 @@ paths:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
/api/v2/auth/refresh:
get:
tags:
- Login
summary: Refresh
description: Refresh the authentication token.
operationId: refresh
parameters:
- name: next
in: query
required: false
schema:
anyOf:
- type: string
- type: 'null'
title: Next
responses:
'200':
description: Successful Response
content:
application/json:
schema: {}
'307':
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPExceptionResponse'
description: Temporary Redirect
'422':
description: Validation Error
content:
application/json:
schema:
$ref: '#/components/schemas/HTTPValidationError'
components:
schemas:
AppBuilderMenuItemResponse:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,23 +65,3 @@ def logout(request: Request) -> RedirectResponse:
)

return response


@auth_router.get(
"/refresh",
responses=create_openapi_http_exception_doc([status.HTTP_307_TEMPORARY_REDIRECT]),
)
def refresh(request: Request, next: None | str = None) -> RedirectResponse:
"""Refresh the authentication token."""
refresh_url = request.app.state.auth_manager.get_url_refresh()

if not refresh_url:
return RedirectResponse(f"{conf.get('api', 'base_url', fallback='/')}auth/logout")

if next and not is_safe_url(next, request=request):
raise HTTPException(status_code=400, detail="Invalid or unsafe next URL")

if next:
refresh_url += f"?next={next}"

return RedirectResponse(refresh_url)
6 changes: 0 additions & 6 deletions airflow-core/src/airflow/ui/openapi-gen/queries/common.ts
Original file line number Diff line number Diff line change
Expand Up @@ -753,12 +753,6 @@ export type LoginServiceLogoutDefaultResponse = Awaited<ReturnType<typeof LoginS
export type LoginServiceLogoutQueryResult<TData = LoginServiceLogoutDefaultResponse, TError = unknown> = UseQueryResult<TData, TError>;
export const useLoginServiceLogoutKey = "LoginServiceLogout";
export const UseLoginServiceLogoutKeyFn = (queryKey?: Array<unknown>) => [useLoginServiceLogoutKey, ...(queryKey ?? [])];
export type LoginServiceRefreshDefaultResponse = Awaited<ReturnType<typeof LoginService.refresh>>;
export type LoginServiceRefreshQueryResult<TData = LoginServiceRefreshDefaultResponse, TError = unknown> = UseQueryResult<TData, TError>;
export const useLoginServiceRefreshKey = "LoginServiceRefresh";
export const UseLoginServiceRefreshKeyFn = ({ next }: {
next?: string;
} = {}, queryKey?: Array<unknown>) => [useLoginServiceRefreshKey, ...(queryKey ?? [{ next }])];
export type AuthLinksServiceGetAuthMenusDefaultResponse = Awaited<ReturnType<typeof AuthLinksService.getAuthMenus>>;
export type AuthLinksServiceGetAuthMenusQueryResult<TData = AuthLinksServiceGetAuthMenusDefaultResponse, TError = unknown> = UseQueryResult<TData, TError>;
export const useAuthLinksServiceGetAuthMenusKey = "AuthLinksServiceGetAuthMenus";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1431,17 +1431,6 @@ export const ensureUseLoginServiceLoginData = (queryClient: QueryClient, { next
*/
export const ensureUseLoginServiceLogoutData = (queryClient: QueryClient) => queryClient.ensureQueryData({ queryKey: Common.UseLoginServiceLogoutKeyFn(), queryFn: () => LoginService.logout() });
/**
* Refresh
* Refresh the authentication token.
* @param data The data for the request.
* @param data.next
* @returns unknown Successful Response
* @throws ApiError
*/
export const ensureUseLoginServiceRefreshData = (queryClient: QueryClient, { next }: {
next?: string;
} = {}) => queryClient.ensureQueryData({ queryKey: Common.UseLoginServiceRefreshKeyFn({ next }), queryFn: () => LoginService.refresh({ next }) });
/**
* Get Auth Menus
* @returns MenuItemCollectionResponse Successful Response
* @throws ApiError
Expand Down
11 changes: 0 additions & 11 deletions airflow-core/src/airflow/ui/openapi-gen/queries/prefetch.ts
Original file line number Diff line number Diff line change
Expand Up @@ -1431,17 +1431,6 @@ export const prefetchUseLoginServiceLogin = (queryClient: QueryClient, { next }:
*/
export const prefetchUseLoginServiceLogout = (queryClient: QueryClient) => queryClient.prefetchQuery({ queryKey: Common.UseLoginServiceLogoutKeyFn(), queryFn: () => LoginService.logout() });
/**
* Refresh
* Refresh the authentication token.
* @param data The data for the request.
* @param data.next
* @returns unknown Successful Response
* @throws ApiError
*/
export const prefetchUseLoginServiceRefresh = (queryClient: QueryClient, { next }: {
next?: string;
} = {}) => queryClient.prefetchQuery({ queryKey: Common.UseLoginServiceRefreshKeyFn({ next }), queryFn: () => LoginService.refresh({ next }) });
/**
* Get Auth Menus
* @returns MenuItemCollectionResponse Successful Response
* @throws ApiError
Expand Down
11 changes: 0 additions & 11 deletions airflow-core/src/airflow/ui/openapi-gen/queries/queries.ts
Original file line number Diff line number Diff line change
Expand Up @@ -1431,17 +1431,6 @@ export const useLoginServiceLogin = <TData = Common.LoginServiceLoginDefaultResp
*/
export const useLoginServiceLogout = <TData = Common.LoginServiceLogoutDefaultResponse, TError = unknown, TQueryKey extends Array<unknown> = unknown[]>(queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">) => useQuery<TData, TError>({ queryKey: Common.UseLoginServiceLogoutKeyFn(queryKey), queryFn: () => LoginService.logout() as TData, ...options });
/**
* Refresh
* Refresh the authentication token.
* @param data The data for the request.
* @param data.next
* @returns unknown Successful Response
* @throws ApiError
*/
export const useLoginServiceRefresh = <TData = Common.LoginServiceRefreshDefaultResponse, TError = unknown, TQueryKey extends Array<unknown> = unknown[]>({ next }: {
next?: string;
} = {}, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">) => useQuery<TData, TError>({ queryKey: Common.UseLoginServiceRefreshKeyFn({ next }, queryKey), queryFn: () => LoginService.refresh({ next }) as TData, ...options });
/**
* Get Auth Menus
* @returns MenuItemCollectionResponse Successful Response
* @throws ApiError
Expand Down
11 changes: 0 additions & 11 deletions airflow-core/src/airflow/ui/openapi-gen/queries/suspense.ts
Original file line number Diff line number Diff line change
Expand Up @@ -1431,17 +1431,6 @@ export const useLoginServiceLoginSuspense = <TData = Common.LoginServiceLoginDef
*/
export const useLoginServiceLogoutSuspense = <TData = Common.LoginServiceLogoutDefaultResponse, TError = unknown, TQueryKey extends Array<unknown> = unknown[]>(queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">) => useSuspenseQuery<TData, TError>({ queryKey: Common.UseLoginServiceLogoutKeyFn(queryKey), queryFn: () => LoginService.logout() as TData, ...options });
/**
* Refresh
* Refresh the authentication token.
* @param data The data for the request.
* @param data.next
* @returns unknown Successful Response
* @throws ApiError
*/
export const useLoginServiceRefreshSuspense = <TData = Common.LoginServiceRefreshDefaultResponse, TError = unknown, TQueryKey extends Array<unknown> = unknown[]>({ next }: {
next?: string;
} = {}, queryKey?: TQueryKey, options?: Omit<UseQueryOptions<TData, TError>, "queryKey" | "queryFn">) => useSuspenseQuery<TData, TError>({ queryKey: Common.UseLoginServiceRefreshKeyFn({ next }, queryKey), queryFn: () => LoginService.refresh({ next }) as TData, ...options });
/**
* Get Auth Menus
* @returns MenuItemCollectionResponse Successful Response
* @throws ApiError
Expand Down
Loading