diff --git a/airflow-core/src/airflow/api_fastapi/common/parameters.py b/airflow-core/src/airflow/api_fastapi/common/parameters.py index 2f60b13b27ee4..914d6284fc5b8 100644 --- a/airflow-core/src/airflow/api_fastapi/common/parameters.py +++ b/airflow-core/src/airflow/api_fastapi/common/parameters.py @@ -32,13 +32,14 @@ overload, ) -from fastapi import Depends, HTTPException, Query, status +from fastapi import Depends, HTTPException, Query from pendulum.parsing.exceptions import ParserError from pydantic import AfterValidator, BaseModel, NonNegativeInt from sqlalchemy import Column, and_, case, func, not_, or_, select as sql_select from sqlalchemy.inspection import inspect from airflow._shared.timezones import timezone +from airflow.api_fastapi.compat import HTTP_422_UNPROCESSABLE_CONTENT from airflow.api_fastapi.core_api.base import OrmClause from airflow.api_fastapi.core_api.security import GetUserDep from airflow.models import Base @@ -779,7 +780,7 @@ def _transform_dag_run_states(states: Iterable[str] | None) -> list[DagRunState return [None if s in ("none", None) else DagRunState(s) for s in states] except ValueError: raise HTTPException( - status_code=status.HTTP_422_UNPROCESSABLE_ENTITY, + status_code=HTTP_422_UNPROCESSABLE_CONTENT, detail=f"Invalid value for state. Valid values are {', '.join(DagRunState)}", ) @@ -805,7 +806,7 @@ def _transform_dag_run_types(types: list[str] | None) -> list[DagRunType | None] return [None if run_type in ("none", None) else DagRunType(run_type) for run_type in types] except ValueError: raise HTTPException( - status_code=status.HTTP_422_UNPROCESSABLE_ENTITY, + status_code=HTTP_422_UNPROCESSABLE_CONTENT, detail=f"Invalid value for run type. Valid values are {', '.join(DagRunType)}", ) @@ -843,7 +844,7 @@ def _transform_ti_states(states: list[str] | None) -> list[TaskInstanceState | N return [None if s in ("no_status", "none", None) else TaskInstanceState(s) for s in states] except ValueError: raise HTTPException( - status_code=status.HTTP_422_UNPROCESSABLE_ENTITY, + status_code=HTTP_422_UNPROCESSABLE_CONTENT, detail=f"Invalid value for state. Valid values are {', '.join(TaskInstanceState)}", ) diff --git a/airflow-core/src/airflow/api_fastapi/compat.py b/airflow-core/src/airflow/api_fastapi/compat.py new file mode 100644 index 0000000000000..06d8822bf9542 --- /dev/null +++ b/airflow-core/src/airflow/api_fastapi/compat.py @@ -0,0 +1,26 @@ +# 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 + +try: + from starlette.status import HTTP_422_UNPROCESSABLE_CONTENT +except ImportError: + from starlette.status import ( # type: ignore[no-redef] + HTTP_422_UNPROCESSABLE_ENTITY as HTTP_422_UNPROCESSABLE_CONTENT, + ) + +__all__ = ["HTTP_422_UNPROCESSABLE_CONTENT"] diff --git a/airflow-core/src/airflow/api_fastapi/core_api/routes/public/dags.py b/airflow-core/src/airflow/api_fastapi/core_api/routes/public/dags.py index 22d6e48d894dd..da2407e361253 100644 --- a/airflow-core/src/airflow/api_fastapi/core_api/routes/public/dags.py +++ b/airflow-core/src/airflow/api_fastapi/core_api/routes/public/dags.py @@ -57,6 +57,7 @@ filter_param_factory, ) from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.compat import HTTP_422_UNPROCESSABLE_CONTENT from airflow.api_fastapi.core_api.datamodels.dags import ( DAGCollectionResponse, DAGDetailsResponse, @@ -178,7 +179,7 @@ def get_dags( [ status.HTTP_400_BAD_REQUEST, status.HTTP_404_NOT_FOUND, - status.HTTP_422_UNPROCESSABLE_ENTITY, + HTTP_422_UNPROCESSABLE_CONTENT, ] ), dependencies=[Depends(requires_access_dag(method="GET"))], @@ -413,7 +414,7 @@ def unfavorite_dag(dag_id: str, session: SessionDep, user: GetUserDep): [ status.HTTP_400_BAD_REQUEST, status.HTTP_404_NOT_FOUND, - status.HTTP_422_UNPROCESSABLE_ENTITY, + HTTP_422_UNPROCESSABLE_CONTENT, ] ), dependencies=[Depends(requires_access_dag(method="DELETE")), Depends(action_logging())], diff --git a/airflow-core/src/airflow/api_fastapi/execution_api/routes/dag_runs.py b/airflow-core/src/airflow/api_fastapi/execution_api/routes/dag_runs.py index b3a99bf21a3db..5eae430a434d1 100644 --- a/airflow-core/src/airflow/api_fastapi/execution_api/routes/dag_runs.py +++ b/airflow-core/src/airflow/api_fastapi/execution_api/routes/dag_runs.py @@ -27,6 +27,7 @@ from airflow.api_fastapi.common.dagbag import DagBagDep, get_dag_for_run from airflow.api_fastapi.common.db.common import SessionDep from airflow.api_fastapi.common.types import UtcDateTime +from airflow.api_fastapi.compat import HTTP_422_UNPROCESSABLE_CONTENT from airflow.api_fastapi.execution_api.datamodels.dagrun import DagRunStateResponse, TriggerDAGRunPayload from airflow.api_fastapi.execution_api.datamodels.taskinstance import DagRun from airflow.exceptions import DagRunAlreadyExists @@ -48,7 +49,7 @@ status.HTTP_400_BAD_REQUEST: {"description": "DAG has import errors and cannot be triggered"}, status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"}, status.HTTP_409_CONFLICT: {"description": "DAG Run already exists for the given dag_id"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload"}, }, ) def trigger_dag_run( @@ -100,7 +101,7 @@ def trigger_dag_run( responses={ status.HTTP_400_BAD_REQUEST: {"description": "DAG has import errors and cannot be triggered"}, status.HTTP_404_NOT_FOUND: {"description": "DAG not found for the given dag_id"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload"}, }, ) def clear_dag_run( diff --git a/airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py b/airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py index 43561cbf1d0ed..56111db979c5e 100644 --- a/airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py +++ b/airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py @@ -41,6 +41,7 @@ from airflow.api_fastapi.common.dagbag import DagBagDep, get_latest_version_of_dag from airflow.api_fastapi.common.db.common import SessionDep from airflow.api_fastapi.common.types import UtcDateTime +from airflow.api_fastapi.compat import HTTP_422_UNPROCESSABLE_CONTENT from airflow.api_fastapi.execution_api.datamodels.taskinstance import ( InactiveAssetsResponse, PrevSuccessfulDagRunResponse, @@ -96,7 +97,7 @@ responses={ status.HTTP_404_NOT_FOUND: {"description": "Task Instance not found"}, status.HTTP_409_CONFLICT: {"description": "The TI is already in the requested state"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload for the state transition"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload for the state transition"}, }, response_model_exclude_unset=True, ) @@ -337,7 +338,7 @@ def _get_upstream_map_indexes( responses={ status.HTTP_404_NOT_FOUND: {"description": "Task Instance not found"}, status.HTTP_409_CONFLICT: {"description": "The TI is already in the requested state"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload for the state transition"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload for the state transition"}, }, ) def ti_update_state( @@ -581,7 +582,7 @@ def _create_ti_state_update_query_and_update_state( status_code=status.HTTP_204_NO_CONTENT, responses={ status.HTTP_404_NOT_FOUND: {"description": "Task Instance not found"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload for the state transition"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload for the state transition"}, }, ) def ti_skip_downstream( @@ -628,7 +629,7 @@ def ti_skip_downstream( status.HTTP_409_CONFLICT: { "description": "The TI attempting to heartbeat should be terminated for the given reason" }, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid payload for the state transition"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid payload for the state transition"}, }, ) def ti_heartbeat( @@ -703,7 +704,7 @@ def ti_heartbeat( # TODO: Do we need to use create_openapi_http_exception_doc here? responses={ status.HTTP_404_NOT_FOUND: {"description": "Task Instance not found"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: { + HTTP_422_UNPROCESSABLE_CONTENT: { "description": "Invalid payload for the setting rendered task instance fields" }, }, @@ -735,7 +736,7 @@ def ti_put_rtif( status_code=status.HTTP_204_NO_CONTENT, responses={ status.HTTP_404_NOT_FOUND: {"description": "Task Instance not found"}, - status.HTTP_422_UNPROCESSABLE_ENTITY: {"description": "Invalid rendered_map_index value"}, + HTTP_422_UNPROCESSABLE_CONTENT: {"description": "Invalid rendered_map_index value"}, }, ) def ti_patch_rendered_map_index( @@ -750,7 +751,7 @@ def ti_patch_rendered_map_index( if not rendered_map_index: log.error("rendered_map_index cannot be empty") raise HTTPException( - status_code=status.HTTP_422_UNPROCESSABLE_ENTITY, + status_code=HTTP_422_UNPROCESSABLE_CONTENT, detail="rendered_map_index cannot be empty", ) diff --git a/providers/fab/src/airflow/providers/fab/auth_manager/fab_auth_manager.py b/providers/fab/src/airflow/providers/fab/auth_manager/fab_auth_manager.py index eaf98737394f0..5c4a9e98d556a 100644 --- a/providers/fab/src/airflow/providers/fab/auth_manager/fab_auth_manager.py +++ b/providers/fab/src/airflow/providers/fab/auth_manager/fab_auth_manager.py @@ -78,6 +78,7 @@ from airflow.providers.fab.www.security.permissions import ( ACTION_CAN_READ, RESOURCE_AUDIT_LOG, + RESOURCE_BACKFILL, RESOURCE_CLUSTER_ACTIVITY, RESOURCE_CONFIG, RESOURCE_CONNECTION, @@ -105,7 +106,6 @@ get_fab_action_from_method_map, get_method_from_fab_action_map, ) -from airflow.security.permissions import RESOURCE_BACKFILL from airflow.utils.session import NEW_SESSION, create_session, provide_session from airflow.utils.yaml import safe_load