diff --git a/airflow/api_connexion/endpoints/config_endpoint.py b/airflow/api_connexion/endpoints/config_endpoint.py index cbb8acdfce166..079fde54637d1 100644 --- a/airflow/api_connexion/endpoints/config_endpoint.py +++ b/airflow/api_connexion/endpoints/config_endpoint.py @@ -25,6 +25,7 @@ from airflow.api_connexion.schemas.config_schema import Config, ConfigOption, ConfigSection, config_schema from airflow.configuration import conf from airflow.settings import json +from airflow.utils.api_migration import mark_fastapi_migration_done LINE_SEP = "\n" # `\n` cannot appear in f-strings @@ -65,6 +66,7 @@ def _config_to_json(config: Config) -> str: return json.dumps(config_schema.dump(config), indent=4) +@mark_fastapi_migration_done @security.requires_access_configuration("GET") def get_config(*, section: str | None = None) -> Response: """Get current configuration.""" @@ -102,6 +104,7 @@ def get_config(*, section: str | None = None) -> Response: ) +@mark_fastapi_migration_done @security.requires_access_configuration("GET") def get_value(*, section: str, option: str) -> Response: serializer = { diff --git a/airflow/api_fastapi/common/headers.py b/airflow/api_fastapi/common/headers.py new file mode 100644 index 0000000000000..7e0bd5aae3d29 --- /dev/null +++ b/airflow/api_fastapi/common/headers.py @@ -0,0 +1,48 @@ +# 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 Depends, Header, HTTPException, status +from typing_extensions import Annotated + +from airflow.api_fastapi.common.types import Mimetype + + +def header_accept_json_or_text_depends( + accept: Annotated[ + str, + Header( + json_schema_extra={ + "type": "string", + "enum": [Mimetype.JSON, Mimetype.TEXT, Mimetype.ANY], + } + ), + ] = Mimetype.ANY, +) -> Mimetype: + if accept.startswith(Mimetype.ANY): + return Mimetype.ANY + if accept.startswith(Mimetype.JSON): + return Mimetype.JSON + if accept.startswith(Mimetype.TEXT): + return Mimetype.TEXT + raise HTTPException( + status_code=status.HTTP_406_NOT_ACCEPTABLE, + detail="Only application/json or text/plain is supported", + ) + + +HeaderAcceptJsonOrText = Annotated[Mimetype, Depends(header_accept_json_or_text_depends)] diff --git a/airflow/api_fastapi/common/types.py b/airflow/api_fastapi/common/types.py index ab10a21c97002..4a41f4f0ceed8 100644 --- a/airflow/api_fastapi/common/types.py +++ b/airflow/api_fastapi/common/types.py @@ -17,6 +17,7 @@ from __future__ import annotations from datetime import timedelta +from enum import Enum from typing import Annotated from pydantic import AfterValidator, AliasGenerator, AwareDatetime, BaseModel, BeforeValidator, ConfigDict @@ -56,3 +57,11 @@ class TimeDelta(BaseModel): TimeDeltaWithValidation = Annotated[TimeDelta, BeforeValidator(_validate_timedelta_field)] + + +class Mimetype(str, Enum): + """Mimetype for the `Content-Type` header.""" + + TEXT = "text/plain" + JSON = "application/json" + ANY = "*/*" diff --git a/airflow/api_fastapi/core_api/datamodels/config.py b/airflow/api_fastapi/core_api/datamodels/config.py new file mode 100644 index 0000000000000..0627832e45f4c --- /dev/null +++ b/airflow/api_fastapi/core_api/datamodels/config.py @@ -0,0 +1,64 @@ +# 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 pydantic import BaseModel + + +class ConfigOption(BaseModel): + """Config option.""" + + key: str + value: str | tuple[str, str] + + @property + def text_format(self): + if isinstance(self.value, tuple): + return f"{self.key} = {self.value[0]} {self.value[1]}" + return f"{self.key} = {self.value}" + + +class ConfigSection(BaseModel): + """Config Section Schema.""" + + name: str + options: list[ConfigOption] + + @property + def text_format(self): + """ + Convert the config section to text format. + + Example: + ``` + [section_name] + key1 = value1 + key2 = value2 + ``` + """ + return f"[{self.name}]\n" + "\n".join(option.text_format for option in self.options) + "\n" + + +class Config(BaseModel): + """List of config sections with their options.""" + + sections: list[ConfigSection] + + @property + def text_format(self): + # convert all config sections to text + return "\n".join(section.text_format for section in self.sections) diff --git a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml index 11c701f953e9b..3c1d85761ec94 100644 --- a/airflow/api_fastapi/core_api/openapi/v1-generated.yaml +++ b/airflow/api_fastapi/core_api/openapi/v1-generated.yaml @@ -1683,6 +1683,10 @@ paths: required: false schema: type: string + enum: + - application/json + - text/plain + - '*/*' default: '*/*' title: Accept responses: @@ -1785,6 +1789,163 @@ paths: application/json: schema: $ref: '#/components/schemas/HTTPValidationError' + /public/config/: + get: + tags: + - Config + summary: Get Config + operationId: get_config + parameters: + - name: section + in: query + required: false + schema: + anyOf: + - type: string + - type: 'null' + title: Section + - name: accept + in: header + required: false + schema: + type: string + enum: + - application/json + - text/plain + - '*/*' + default: '*/*' + title: Accept + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/Config' + text/plain: + schema: + type: string + example: '[core] + + dags_folder = /opt/airflow/dags + + base_log_folder = /opt/airflow/logs + + + [smtp] + + smtp_host = localhost + + smtp_mail_from = airflow@example.com + + ' + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '406': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Acceptable + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' + /public/config/section/{section}/option/{option}: + get: + tags: + - Config + summary: Get Config Value + operationId: get_config_value + parameters: + - name: section + in: path + required: true + schema: + type: string + title: Section + - name: option + in: path + required: true + schema: + type: string + title: Option + - name: accept + in: header + required: false + schema: + type: string + enum: + - application/json + - text/plain + - '*/*' + default: '*/*' + title: Accept + responses: + '200': + description: Successful Response + content: + application/json: + schema: + $ref: '#/components/schemas/Config' + text/plain: + schema: + type: string + example: '[core] + + dags_folder = /opt/airflow/dags + + base_log_folder = /opt/airflow/logs + + ' + '401': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Unauthorized + '403': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Forbidden + '404': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Found + '406': + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPExceptionResponse' + description: Not Acceptable + '422': + description: Validation Error + content: + application/json: + schema: + $ref: '#/components/schemas/HTTPValidationError' /public/dagWarnings: get: tags: @@ -4593,6 +4754,55 @@ components: - status title: BaseInfoSchema description: Base status field for metadatabase and scheduler. + Config: + properties: + sections: + items: + $ref: '#/components/schemas/ConfigSection' + type: array + title: Sections + type: object + required: + - sections + title: Config + description: List of config sections with their options. + ConfigOption: + properties: + key: + type: string + title: Key + value: + anyOf: + - type: string + - prefixItems: + - type: string + - type: string + type: array + maxItems: 2 + minItems: 2 + title: Value + type: object + required: + - key + - value + title: ConfigOption + description: Config option. + ConfigSection: + properties: + name: + type: string + title: Name + options: + items: + $ref: '#/components/schemas/ConfigOption' + type: array + title: Options + type: object + required: + - name + - options + title: ConfigSection + description: Config Section Schema. ConnectionBody: properties: connection_id: diff --git a/airflow/api_fastapi/core_api/routes/public/__init__.py b/airflow/api_fastapi/core_api/routes/public/__init__.py index e85d17ae4ca84..ae4035f6d3532 100644 --- a/airflow/api_fastapi/core_api/routes/public/__init__.py +++ b/airflow/api_fastapi/core_api/routes/public/__init__.py @@ -23,6 +23,7 @@ from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc from airflow.api_fastapi.core_api.routes.public.assets import assets_router from airflow.api_fastapi.core_api.routes.public.backfills import backfills_router +from airflow.api_fastapi.core_api.routes.public.config import config_router from airflow.api_fastapi.core_api.routes.public.connections import connections_router from airflow.api_fastapi.core_api.routes.public.dag_run import dag_run_router from airflow.api_fastapi.core_api.routes.public.dag_sources import dag_sources_router @@ -54,6 +55,7 @@ authenticated_router.include_router(dag_run_router) authenticated_router.include_router(dag_sources_router) authenticated_router.include_router(dag_stats_router) +authenticated_router.include_router(config_router) authenticated_router.include_router(dag_warning_router) authenticated_router.include_router(dags_router) authenticated_router.include_router(event_logs_router) diff --git a/airflow/api_fastapi/core_api/routes/public/config.py b/airflow/api_fastapi/core_api/routes/public/config.py new file mode 100644 index 0000000000000..eb26cfcd2c02a --- /dev/null +++ b/airflow/api_fastapi/core_api/routes/public/config.py @@ -0,0 +1,174 @@ +# 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 textwrap + +from fastapi import HTTPException, status +from fastapi.responses import Response + +from airflow.api_fastapi.common.headers import HeaderAcceptJsonOrText +from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.common.types import Mimetype +from airflow.api_fastapi.core_api.datamodels.config import ( + Config, + ConfigOption, + ConfigSection, +) +from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc +from airflow.configuration import conf + +text_example_response_for_get_config_value = { + Mimetype.TEXT: { + "schema": { + "type": "string", + "example": textwrap.dedent( + """\ + [core] + dags_folder = /opt/airflow/dags + base_log_folder = /opt/airflow/logs + """ + ), + } + } +} + +text_example_response_for_get_config = { + Mimetype.TEXT: { + "schema": { + "type": "string", + "example": textwrap.dedent( + """\ + [core] + dags_folder = /opt/airflow/dags + base_log_folder = /opt/airflow/logs + + [smtp] + smtp_host = localhost + smtp_mail_from = airflow@example.com + """ + ), + }, + } +} + + +def _check_expose_config() -> bool: + display_sensitive: bool | None = None + if conf.get("webserver", "expose_config").lower() == "non-sensitive-only": + expose_config = True + display_sensitive = False + else: + expose_config = conf.getboolean("webserver", "expose_config") + display_sensitive = True + + if not expose_config: + raise HTTPException( + status_code=status.HTTP_403_FORBIDDEN, + detail="Your Airflow administrator chose not to expose the configuration, most likely for security reasons.", + ) + return display_sensitive + + +def _response_based_on_accept(accept: Mimetype, config: Config): + if accept == Mimetype.TEXT: + return Response(content=config.text_format, media_type=Mimetype.TEXT) + return config + + +config_router = AirflowRouter(tags=["Config"], prefix="/config") + + +@config_router.get( + "/", + responses={ + **create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_406_NOT_ACCEPTABLE, + ] + ), + "200": {"description": "Successful Response", "content": text_example_response_for_get_config}, + }, + response_model=Config, +) +def get_config( + accept: HeaderAcceptJsonOrText, + section: str | None = None, +): + display_sensitive = _check_expose_config() + + if section and not conf.has_section(section): + raise HTTPException( + status_code=status.HTTP_404_NOT_FOUND, + detail=f"Section {section} not found.", + ) + conf_dict = conf.as_dict(display_source=False, display_sensitive=display_sensitive) + + if section: + conf_section_value = conf_dict[section] + conf_dict.clear() + conf_dict[section] = conf_section_value + + config = Config( + sections=[ + ConfigSection( + name=section, options=[ConfigOption(key=key, value=value) for key, value in options.items()] + ) + for section, options in conf_dict.items() + ] + ) + return _response_based_on_accept(accept, config) + + +@config_router.get( + "/section/{section}/option/{option}", + responses={ + **create_openapi_http_exception_doc( + [ + status.HTTP_401_UNAUTHORIZED, + status.HTTP_403_FORBIDDEN, + status.HTTP_404_NOT_FOUND, + status.HTTP_406_NOT_ACCEPTABLE, + ] + ), + "200": {"description": "Successful Response", "content": text_example_response_for_get_config_value}, + }, + response_model=Config, +) +def get_config_value( + section: str, + option: str, + accept: HeaderAcceptJsonOrText, +): + _check_expose_config() + + if not conf.has_option(section, option): + raise HTTPException( + status_code=status.HTTP_404_NOT_FOUND, + detail=f"Option [{section}/{option}] not found.", + ) + + if (section.lower(), option.lower()) in conf.sensitive_config_values: + value = "< hidden >" + else: + value = conf.get(section, option) + + config = Config(sections=[ConfigSection(name=section, options=[ConfigOption(key=option, value=value)])]) + return _response_based_on_accept(accept, config) diff --git a/airflow/api_fastapi/core_api/routes/public/dag_sources.py b/airflow/api_fastapi/core_api/routes/public/dag_sources.py index fbe30620228b5..d28b08ea8bc88 100644 --- a/airflow/api_fastapi/core_api/routes/public/dag_sources.py +++ b/airflow/api_fastapi/core_api/routes/public/dag_sources.py @@ -18,21 +18,19 @@ from typing import Annotated -from fastapi import Depends, Header, HTTPException, Request, Response, status +from fastapi import Depends, HTTPException, Response, status from sqlalchemy.orm import Session from airflow.api_fastapi.common.db.common import get_session +from airflow.api_fastapi.common.headers import HeaderAcceptJsonOrText from airflow.api_fastapi.common.router import AirflowRouter +from airflow.api_fastapi.common.types import Mimetype from airflow.api_fastapi.core_api.datamodels.dag_sources import DAGSourceResponse from airflow.api_fastapi.core_api.openapi.exceptions import create_openapi_http_exception_doc from airflow.models.dag_version import DagVersion dag_sources_router = AirflowRouter(tags=["DagSource"], prefix="/dagSources") -mime_type_text = "text/plain" -mime_type_json = "application/json" -mime_type_any = "*/*" - @dag_sources_router.get( "/{dag_id}", @@ -47,17 +45,16 @@ "200": { "description": "Successful Response", "content": { - mime_type_text: {"schema": {"type": "string", "example": "dag code"}}, + Mimetype.TEXT: {"schema": {"type": "string", "example": "dag code"}}, }, }, }, response_model=DAGSourceResponse, ) def get_dag_source( + accept: HeaderAcceptJsonOrText, dag_id: str, session: Annotated[Session, Depends(get_session)], - request: Request, - accept: Annotated[str, Header()] = mime_type_any, version_number: int | None = None, ): """Get source code using file token.""" @@ -72,8 +69,6 @@ def get_dag_source( version_number = dag_version.version_number dag_source_model = DAGSourceResponse(dag_id=dag_id, content=dag_source, version_number=version_number) - if accept.startswith(mime_type_text): - return Response(dag_source_model.content, media_type=mime_type_text) - if accept.startswith(mime_type_json) or accept.startswith(mime_type_any): - return dag_source_model - raise HTTPException(status.HTTP_406_NOT_ACCEPTABLE, "Content not available for Accept header") + if accept == Mimetype.TEXT: + return Response(dag_source_model.content, media_type=Mimetype.TEXT) + return dag_source_model diff --git a/airflow/ui/openapi-gen/queries/common.ts b/airflow/ui/openapi-gen/queries/common.ts index 3d52ef341f31e..ea4e65d44c91d 100644 --- a/airflow/ui/openapi-gen/queries/common.ts +++ b/airflow/ui/openapi-gen/queries/common.ts @@ -4,6 +4,7 @@ import { UseQueryResult } from "@tanstack/react-query"; import { AssetService, BackfillService, + ConfigService, ConnectionService, DagRunService, DagService, @@ -410,7 +411,7 @@ export const UseDagSourceServiceGetDagSourceKeyFn = ( dagId, versionNumber, }: { - accept?: string; + accept?: "application/json" | "text/plain" | "*/*"; dagId: string; versionNumber?: number; }, @@ -435,6 +436,47 @@ export const UseDagStatsServiceGetDagStatsKeyFn = ( } = {}, queryKey?: Array, ) => [useDagStatsServiceGetDagStatsKey, ...(queryKey ?? [{ dagIds }])]; +export type ConfigServiceGetConfigDefaultResponse = Awaited< + ReturnType +>; +export type ConfigServiceGetConfigQueryResult< + TData = ConfigServiceGetConfigDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useConfigServiceGetConfigKey = "ConfigServiceGetConfig"; +export const UseConfigServiceGetConfigKeyFn = ( + { + accept, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + section?: string; + } = {}, + queryKey?: Array, +) => [useConfigServiceGetConfigKey, ...(queryKey ?? [{ accept, section }])]; +export type ConfigServiceGetConfigValueDefaultResponse = Awaited< + ReturnType +>; +export type ConfigServiceGetConfigValueQueryResult< + TData = ConfigServiceGetConfigValueDefaultResponse, + TError = unknown, +> = UseQueryResult; +export const useConfigServiceGetConfigValueKey = "ConfigServiceGetConfigValue"; +export const UseConfigServiceGetConfigValueKeyFn = ( + { + accept, + option, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + option: string; + section: string; + }, + queryKey?: Array, +) => [ + useConfigServiceGetConfigValueKey, + ...(queryKey ?? [{ accept, option, section }]), +]; export type DagWarningServiceListDagWarningsDefaultResponse = Awaited< ReturnType >; diff --git a/airflow/ui/openapi-gen/queries/prefetch.ts b/airflow/ui/openapi-gen/queries/prefetch.ts index baf49a781b5c5..273882c787dcb 100644 --- a/airflow/ui/openapi-gen/queries/prefetch.ts +++ b/airflow/ui/openapi-gen/queries/prefetch.ts @@ -4,6 +4,7 @@ import { type QueryClient } from "@tanstack/react-query"; import { AssetService, BackfillService, + ConfigService, ConnectionService, DagRunService, DagService, @@ -508,7 +509,7 @@ export const prefetchUseDagSourceServiceGetDagSource = ( dagId, versionNumber, }: { - accept?: string; + accept?: "application/json" | "text/plain" | "*/*"; dagId: string; versionNumber?: number; }, @@ -542,6 +543,57 @@ export const prefetchUseDagStatsServiceGetDagStats = ( queryKey: Common.UseDagStatsServiceGetDagStatsKeyFn({ dagIds }), queryFn: () => DagStatsService.getDagStats({ dagIds }), }); +/** + * Get Config + * @param data The data for the request. + * @param data.section + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const prefetchUseConfigServiceGetConfig = ( + queryClient: QueryClient, + { + accept, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + section?: string; + } = {}, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseConfigServiceGetConfigKeyFn({ accept, section }), + queryFn: () => ConfigService.getConfig({ accept, section }), + }); +/** + * Get Config Value + * @param data The data for the request. + * @param data.section + * @param data.option + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const prefetchUseConfigServiceGetConfigValue = ( + queryClient: QueryClient, + { + accept, + option, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + option: string; + section: string; + }, +) => + queryClient.prefetchQuery({ + queryKey: Common.UseConfigServiceGetConfigValueKeyFn({ + accept, + option, + section, + }), + queryFn: () => ConfigService.getConfigValue({ accept, option, section }), + }); /** * List Dag Warnings * Get a list of DAG warnings. diff --git a/airflow/ui/openapi-gen/queries/queries.ts b/airflow/ui/openapi-gen/queries/queries.ts index 6a228e5f520f5..386575204aa0b 100644 --- a/airflow/ui/openapi-gen/queries/queries.ts +++ b/airflow/ui/openapi-gen/queries/queries.ts @@ -9,6 +9,7 @@ import { import { AssetService, BackfillService, + ConfigService, ConnectionService, DagRunService, DagService, @@ -641,7 +642,7 @@ export const useDagSourceServiceGetDagSource = < dagId, versionNumber, }: { - accept?: string; + accept?: "application/json" | "text/plain" | "*/*"; dagId: string; versionNumber?: number; }, @@ -683,6 +684,72 @@ export const useDagStatsServiceGetDagStats = < queryFn: () => DagStatsService.getDagStats({ dagIds }) as TData, ...options, }); +/** + * Get Config + * @param data The data for the request. + * @param data.section + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const useConfigServiceGetConfig = < + TData = Common.ConfigServiceGetConfigDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + section?: string; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseConfigServiceGetConfigKeyFn( + { accept, section }, + queryKey, + ), + queryFn: () => ConfigService.getConfig({ accept, section }) as TData, + ...options, + }); +/** + * Get Config Value + * @param data The data for the request. + * @param data.section + * @param data.option + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const useConfigServiceGetConfigValue = < + TData = Common.ConfigServiceGetConfigValueDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + option, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + option: string; + section: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useQuery({ + queryKey: Common.UseConfigServiceGetConfigValueKeyFn( + { accept, option, section }, + queryKey, + ), + queryFn: () => + ConfigService.getConfigValue({ accept, option, section }) as TData, + ...options, + }); /** * List Dag Warnings * Get a list of DAG warnings. diff --git a/airflow/ui/openapi-gen/queries/suspense.ts b/airflow/ui/openapi-gen/queries/suspense.ts index d663543cfd9ca..3d57a08a69e64 100644 --- a/airflow/ui/openapi-gen/queries/suspense.ts +++ b/airflow/ui/openapi-gen/queries/suspense.ts @@ -4,6 +4,7 @@ import { UseQueryOptions, useSuspenseQuery } from "@tanstack/react-query"; import { AssetService, BackfillService, + ConfigService, ConnectionService, DagRunService, DagService, @@ -623,7 +624,7 @@ export const useDagSourceServiceGetDagSourceSuspense = < dagId, versionNumber, }: { - accept?: string; + accept?: "application/json" | "text/plain" | "*/*"; dagId: string; versionNumber?: number; }, @@ -665,6 +666,72 @@ export const useDagStatsServiceGetDagStatsSuspense = < queryFn: () => DagStatsService.getDagStats({ dagIds }) as TData, ...options, }); +/** + * Get Config + * @param data The data for the request. + * @param data.section + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const useConfigServiceGetConfigSuspense = < + TData = Common.ConfigServiceGetConfigDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + section?: string; + } = {}, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseConfigServiceGetConfigKeyFn( + { accept, section }, + queryKey, + ), + queryFn: () => ConfigService.getConfig({ accept, section }) as TData, + ...options, + }); +/** + * Get Config Value + * @param data The data for the request. + * @param data.section + * @param data.option + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ +export const useConfigServiceGetConfigValueSuspense = < + TData = Common.ConfigServiceGetConfigValueDefaultResponse, + TError = unknown, + TQueryKey extends Array = unknown[], +>( + { + accept, + option, + section, + }: { + accept?: "application/json" | "text/plain" | "*/*"; + option: string; + section: string; + }, + queryKey?: TQueryKey, + options?: Omit, "queryKey" | "queryFn">, +) => + useSuspenseQuery({ + queryKey: Common.UseConfigServiceGetConfigValueKeyFn( + { accept, option, section }, + queryKey, + ), + queryFn: () => + ConfigService.getConfigValue({ accept, option, section }) as TData, + ...options, + }); /** * List Dag Warnings * Get a list of DAG warnings. diff --git a/airflow/ui/openapi-gen/requests/schemas.gen.ts b/airflow/ui/openapi-gen/requests/schemas.gen.ts index 2e4a9cbc832c4..0f08034c533dd 100644 --- a/airflow/ui/openapi-gen/requests/schemas.gen.ts +++ b/airflow/ui/openapi-gen/requests/schemas.gen.ts @@ -459,6 +459,76 @@ export const $BaseInfoSchema = { description: "Base status field for metadatabase and scheduler.", } as const; +export const $Config = { + properties: { + sections: { + items: { + $ref: "#/components/schemas/ConfigSection", + }, + type: "array", + title: "Sections", + }, + }, + type: "object", + required: ["sections"], + title: "Config", + description: "List of config sections with their options.", +} as const; + +export const $ConfigOption = { + properties: { + key: { + type: "string", + title: "Key", + }, + value: { + anyOf: [ + { + type: "string", + }, + { + prefixItems: [ + { + type: "string", + }, + { + type: "string", + }, + ], + type: "array", + maxItems: 2, + minItems: 2, + }, + ], + title: "Value", + }, + }, + type: "object", + required: ["key", "value"], + title: "ConfigOption", + description: "Config option.", +} as const; + +export const $ConfigSection = { + properties: { + name: { + type: "string", + title: "Name", + }, + options: { + items: { + $ref: "#/components/schemas/ConfigOption", + }, + type: "array", + title: "Options", + }, + }, + type: "object", + required: ["name", "options"], + title: "ConfigSection", + description: "Config Section Schema.", +} as const; + export const $ConnectionBody = { properties: { connection_id: { diff --git a/airflow/ui/openapi-gen/requests/services.gen.ts b/airflow/ui/openapi-gen/requests/services.gen.ts index 72ee239974201..6689fce3a837d 100644 --- a/airflow/ui/openapi-gen/requests/services.gen.ts +++ b/airflow/ui/openapi-gen/requests/services.gen.ts @@ -67,6 +67,10 @@ import type { GetDagSourceResponse, GetDagStatsData, GetDagStatsResponse, + GetConfigData, + GetConfigResponse, + GetConfigValueData, + GetConfigValueResponse, ListDagWarningsData, ListDagWarningsResponse, GetDagsData, @@ -1096,6 +1100,70 @@ export class DagStatsService { } } +export class ConfigService { + /** + * Get Config + * @param data The data for the request. + * @param data.section + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ + public static getConfig( + data: GetConfigData = {}, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/config/", + headers: { + accept: data.accept, + }, + query: { + section: data.section, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 406: "Not Acceptable", + 422: "Validation Error", + }, + }); + } + + /** + * Get Config Value + * @param data The data for the request. + * @param data.section + * @param data.option + * @param data.accept + * @returns Config Successful Response + * @throws ApiError + */ + public static getConfigValue( + data: GetConfigValueData, + ): CancelablePromise { + return __request(OpenAPI, { + method: "GET", + url: "/public/config/section/{section}/option/{option}", + path: { + section: data.section, + option: data.option, + }, + headers: { + accept: data.accept, + }, + errors: { + 401: "Unauthorized", + 403: "Forbidden", + 404: "Not Found", + 406: "Not Acceptable", + 422: "Validation Error", + }, + }); + } +} + export class DagWarningService { /** * List Dag Warnings diff --git a/airflow/ui/openapi-gen/requests/types.gen.ts b/airflow/ui/openapi-gen/requests/types.gen.ts index 50fb44a057a25..27e3546e5c7be 100644 --- a/airflow/ui/openapi-gen/requests/types.gen.ts +++ b/airflow/ui/openapi-gen/requests/types.gen.ts @@ -128,6 +128,29 @@ export type BaseInfoSchema = { status: string | null; }; +/** + * List of config sections with their options. + */ +export type Config = { + sections: Array; +}; + +/** + * Config option. + */ +export type ConfigOption = { + key: string; + value: string | [string, string]; +}; + +/** + * Config Section Schema. + */ +export type ConfigSection = { + name: string; + options: Array; +}; + /** * Connection Serializer for requests body. */ @@ -1278,7 +1301,7 @@ export type ClearDagRunResponse = | DAGRunResponse; export type GetDagSourceData = { - accept?: string; + accept?: "application/json" | "text/plain" | "*/*"; dagId: string; versionNumber?: number | null; }; @@ -1291,6 +1314,21 @@ export type GetDagStatsData = { export type GetDagStatsResponse = DagStatsCollectionResponse; +export type GetConfigData = { + accept?: "application/json" | "text/plain" | "*/*"; + section?: string | null; +}; + +export type GetConfigResponse = Config; + +export type GetConfigValueData = { + accept?: "application/json" | "text/plain" | "*/*"; + option: string; + section: string; +}; + +export type GetConfigValueResponse = Config; + export type ListDagWarningsData = { dagId?: string | null; limit?: number; @@ -2476,6 +2514,68 @@ export type $OpenApiTs = { }; }; }; + "/public/config/": { + get: { + req: GetConfigData; + res: { + /** + * Successful Response + */ + 200: Config; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Not Acceptable + */ + 406: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; + "/public/config/section/{section}/option/{option}": { + get: { + req: GetConfigValueData; + res: { + /** + * Successful Response + */ + 200: Config; + /** + * Unauthorized + */ + 401: HTTPExceptionResponse; + /** + * Forbidden + */ + 403: HTTPExceptionResponse; + /** + * Not Found + */ + 404: HTTPExceptionResponse; + /** + * Not Acceptable + */ + 406: HTTPExceptionResponse; + /** + * Validation Error + */ + 422: HTTPValidationError; + }; + }; + }; "/public/dagWarnings": { get: { req: ListDagWarningsData; diff --git a/tests/api_fastapi/core_api/routes/public/test_config.py b/tests/api_fastapi/core_api/routes/public/test_config.py new file mode 100644 index 0000000000000..76aa5b0b6a531 --- /dev/null +++ b/tests/api_fastapi/core_api/routes/public/test_config.py @@ -0,0 +1,474 @@ +# 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 textwrap +from typing import Generator +from unittest.mock import patch + +import pytest + +from tests_common.test_utils.config import conf_vars + +HEADERS_NONE = None +HEADERS_ANY = {"Accept": "*/*"} +HEADERS_JSON = {"Accept": "application/json"} +HEADERS_TEXT = {"Accept": "text/plain"} +HEADERS_INVALID = {"Accept": "invalid"} +HEADERS_JSON_UTF8 = {"Accept": "application/json; charset=utf-8"} +SECTION_CORE = "core" +SECTION_SMTP = "smtp" +SECTION_DATABASE = "database" +SECTION_NOT_EXIST = "not_exist_section" +OPTION_KEY_PARALLELISM = "parallelism" +OPTION_KEY_SMTP_HOST = "smtp_host" +OPTION_KEY_SMTP_MAIL_FROM = "smtp_mail_from" +OPTION_KEY_SQL_ALCHEMY_CONN = "sql_alchemy_conn" +OPTION_VALUE_PARALLELISM = "1024" +OPTION_VALUE_SMTP_HOST = "smtp.example.com" +OPTION_VALUE_SMTP_MAIL_FROM = "airflow@example.com" +OPTION_VALUE_SQL_ALCHEMY_CONN = "sqlite:///example.db" +OPTION_NOT_EXIST = "not_exist_option" +OPTION_VALUE_SENSITIVE_HIDDEN = "< hidden >" + +MOCK_CONFIG_DICT = { + SECTION_CORE: { + OPTION_KEY_PARALLELISM: OPTION_VALUE_PARALLELISM, + }, + SECTION_SMTP: { + OPTION_KEY_SMTP_HOST: OPTION_VALUE_SMTP_HOST, + OPTION_KEY_SMTP_MAIL_FROM: OPTION_VALUE_SMTP_MAIL_FROM, + }, + SECTION_DATABASE: { + OPTION_KEY_SQL_ALCHEMY_CONN: OPTION_VALUE_SQL_ALCHEMY_CONN, + }, +} +MOCK_CONFIG_DICT_SENSITIVE_HIDDEN = { + SECTION_CORE: { + OPTION_KEY_PARALLELISM: OPTION_VALUE_PARALLELISM, + }, + SECTION_SMTP: { + OPTION_KEY_SMTP_HOST: OPTION_VALUE_SMTP_HOST, + OPTION_KEY_SMTP_MAIL_FROM: OPTION_VALUE_SMTP_MAIL_FROM, + }, + SECTION_DATABASE: { + OPTION_KEY_SQL_ALCHEMY_CONN: OPTION_VALUE_SENSITIVE_HIDDEN, + }, +} +MOCK_CONFIG_OVERRIDE = { + (SECTION_CORE, OPTION_KEY_PARALLELISM): OPTION_VALUE_PARALLELISM, + (SECTION_SMTP, OPTION_KEY_SMTP_HOST): OPTION_VALUE_SMTP_HOST, + (SECTION_SMTP, OPTION_KEY_SMTP_MAIL_FROM): OPTION_VALUE_SMTP_MAIL_FROM, +} + +AIRFLOW_CONFIG_ENABLE_EXPOSE_CONFIG = {("webserver", "expose_config"): "True"} +AIRFLOW_CONFIG_DISABLE_EXPOSE_CONFIG = {("webserver", "expose_config"): "False"} +AIRFLOW_CONFIG_NON_SENSITIVE_ONLY_CONFIG = {("webserver", "expose_config"): "non-sensitive-only"} +FORBIDDEN_RESPONSE = { + "detail": "Your Airflow administrator chose not to expose the configuration, most likely for security reasons." +} + +GET_CONFIG_ALL_JSON_RESPONSE = { + "sections": [ + { + "name": SECTION_CORE, + "options": [ + {"key": OPTION_KEY_PARALLELISM, "value": OPTION_VALUE_PARALLELISM}, + ], + }, + { + "name": SECTION_SMTP, + "options": [ + {"key": OPTION_KEY_SMTP_HOST, "value": OPTION_VALUE_SMTP_HOST}, + {"key": OPTION_KEY_SMTP_MAIL_FROM, "value": OPTION_VALUE_SMTP_MAIL_FROM}, + ], + }, + { + "name": SECTION_DATABASE, + "options": [ + {"key": OPTION_KEY_SQL_ALCHEMY_CONN, "value": OPTION_VALUE_SQL_ALCHEMY_CONN}, + ], + }, + ], +} +GET_CONFIG_NON_SENSITIVE_ONLY_JSON_RESPONSE = { + "sections": [ + { + "name": SECTION_CORE, + "options": [ + {"key": OPTION_KEY_PARALLELISM, "value": OPTION_VALUE_PARALLELISM}, + ], + }, + { + "name": SECTION_SMTP, + "options": [ + {"key": OPTION_KEY_SMTP_HOST, "value": OPTION_VALUE_SMTP_HOST}, + {"key": OPTION_KEY_SMTP_MAIL_FROM, "value": OPTION_VALUE_SMTP_MAIL_FROM}, + ], + }, + { + "name": SECTION_DATABASE, + "options": [ + {"key": OPTION_KEY_SQL_ALCHEMY_CONN, "value": OPTION_VALUE_SENSITIVE_HIDDEN}, + ], + }, + ], +} +GET_CONFIG_VALUE_CORE_PARALLELISM_JSON_RESPONSE = { + "sections": [ + { + "name": SECTION_CORE, + "options": [ + {"key": OPTION_KEY_PARALLELISM, "value": OPTION_VALUE_PARALLELISM}, + ], + }, + ], +} +GET_CONFIG_VALUE_NON_SENSITIVE_ONLY_DATABASE_SQL_ALCHEMY_CONN_JSON_RESPONSE = { + "sections": [ + { + "name": SECTION_DATABASE, + "options": [ + {"key": OPTION_KEY_SQL_ALCHEMY_CONN, "value": OPTION_VALUE_SENSITIVE_HIDDEN}, + ], + }, + ], +} + + +class TestConfigEndpoint: + def _validate_response(self, headers, expected_response, expected_status_code, response): + assert response.status_code == expected_status_code + if headers == HEADERS_TEXT: + assert response.text == expected_response + else: + assert response.json() == expected_response + + @pytest.fixture(autouse=True) + def setup(self) -> Generator[None, None, None]: + with conf_vars(AIRFLOW_CONFIG_ENABLE_EXPOSE_CONFIG | MOCK_CONFIG_OVERRIDE): + # since the endpoint calls `conf_dict.clear()` to remove extra keys, + # use `new` instead of `return_value` to avoid side effects + def _mock_conf_as_dict(display_sensitive: bool, **_): + return ( + MOCK_CONFIG_DICT_SENSITIVE_HIDDEN.copy() + if not display_sensitive + else MOCK_CONFIG_DICT.copy() + ) + + with patch( + "airflow.api_fastapi.core_api.routes.public.config.conf.as_dict", + new=_mock_conf_as_dict, + ): + yield + + +class TestGetConfig(TestConfigEndpoint): + @pytest.mark.parametrize( + "section, headers, expected_status_code, expected_response", + [ + ( + None, + HEADERS_JSON, + 200, + GET_CONFIG_ALL_JSON_RESPONSE, + ), + (None, HEADERS_JSON_UTF8, 200, GET_CONFIG_ALL_JSON_RESPONSE), + (None, HEADERS_ANY, 200, GET_CONFIG_ALL_JSON_RESPONSE), + (None, HEADERS_NONE, 200, GET_CONFIG_ALL_JSON_RESPONSE), + ( + None, + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_CORE}] + {OPTION_KEY_PARALLELISM} = {OPTION_VALUE_PARALLELISM} + + [{SECTION_SMTP}] + {OPTION_KEY_SMTP_HOST} = {OPTION_VALUE_SMTP_HOST} + {OPTION_KEY_SMTP_MAIL_FROM} = {OPTION_VALUE_SMTP_MAIL_FROM} + + [{SECTION_DATABASE}] + {OPTION_KEY_SQL_ALCHEMY_CONN} = {OPTION_VALUE_SQL_ALCHEMY_CONN} + """ + ), + ), + ( + None, + HEADERS_INVALID, + 406, + {"detail": "Only application/json or text/plain is supported"}, + ), + ( + SECTION_CORE, + HEADERS_JSON, + 200, + { + "sections": [ + { + "name": SECTION_CORE, + "options": [ + {"key": OPTION_KEY_PARALLELISM, "value": OPTION_VALUE_PARALLELISM}, + ], + }, + ], + }, + ), + ( + SECTION_SMTP, + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_SMTP}] + {OPTION_KEY_SMTP_HOST} = {OPTION_VALUE_SMTP_HOST} + {OPTION_KEY_SMTP_MAIL_FROM} = {OPTION_VALUE_SMTP_MAIL_FROM} + """ + ), + ), + ( + SECTION_DATABASE, + HEADERS_JSON, + 200, + { + "sections": [ + { + "name": SECTION_DATABASE, + "options": [ + {"key": OPTION_KEY_SQL_ALCHEMY_CONN, "value": OPTION_VALUE_SQL_ALCHEMY_CONN}, + ], + }, + ], + }, + ), + (None, HEADERS_JSON, 403, FORBIDDEN_RESPONSE), + (SECTION_CORE, HEADERS_JSON, 403, FORBIDDEN_RESPONSE), + (SECTION_NOT_EXIST, HEADERS_JSON, 404, {"detail": f"Section {SECTION_NOT_EXIST} not found."}), + ], + ) + def test_get_config(self, test_client, section, headers, expected_status_code, expected_response): + query_params = {"section": section} if section else None + if expected_status_code == 403: + with conf_vars(AIRFLOW_CONFIG_DISABLE_EXPOSE_CONFIG): + response = test_client.get("/public/config/", headers=headers, params=query_params) + else: + response = test_client.get("/public/config/", headers=headers, params=query_params) + self._validate_response(headers, expected_response, expected_status_code, response) + + @pytest.mark.parametrize( + "headers, expected_status_code, expected_response", + [ + (HEADERS_JSON, 200, GET_CONFIG_NON_SENSITIVE_ONLY_JSON_RESPONSE), + (HEADERS_JSON_UTF8, 200, GET_CONFIG_NON_SENSITIVE_ONLY_JSON_RESPONSE), + (HEADERS_ANY, 200, GET_CONFIG_NON_SENSITIVE_ONLY_JSON_RESPONSE), + (HEADERS_NONE, 200, GET_CONFIG_NON_SENSITIVE_ONLY_JSON_RESPONSE), + ( + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_CORE}] + {OPTION_KEY_PARALLELISM} = {OPTION_VALUE_PARALLELISM} + + [{SECTION_SMTP}] + {OPTION_KEY_SMTP_HOST} = {OPTION_VALUE_SMTP_HOST} + {OPTION_KEY_SMTP_MAIL_FROM} = {OPTION_VALUE_SMTP_MAIL_FROM} + + [{SECTION_DATABASE}] + {OPTION_KEY_SQL_ALCHEMY_CONN} = {OPTION_VALUE_SENSITIVE_HIDDEN} + """ + ), + ), + ], + ) + def test_get_config_non_sensitive_only( + self, test_client, headers, expected_status_code, expected_response + ): + with conf_vars(AIRFLOW_CONFIG_NON_SENSITIVE_ONLY_CONFIG): + response = test_client.get("/public/config/", headers=headers) + self._validate_response(headers, expected_response, expected_status_code, response) + + +class TestGetConfigValue(TestConfigEndpoint): + @pytest.mark.parametrize( + "section, option, headers, expected_status_code, expected_response", + [ + ( + SECTION_CORE, + OPTION_KEY_PARALLELISM, + HEADERS_JSON, + 200, + GET_CONFIG_VALUE_CORE_PARALLELISM_JSON_RESPONSE, + ), + ( + SECTION_CORE, + OPTION_KEY_PARALLELISM, + HEADERS_JSON_UTF8, + 200, + GET_CONFIG_VALUE_CORE_PARALLELISM_JSON_RESPONSE, + ), + ( + SECTION_CORE, + OPTION_KEY_PARALLELISM, + HEADERS_ANY, + 200, + GET_CONFIG_VALUE_CORE_PARALLELISM_JSON_RESPONSE, + ), + ( + SECTION_CORE, + OPTION_KEY_PARALLELISM, + HEADERS_NONE, + 200, + GET_CONFIG_VALUE_CORE_PARALLELISM_JSON_RESPONSE, + ), + ( + SECTION_SMTP, + OPTION_KEY_SMTP_HOST, + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_SMTP}] + {OPTION_KEY_SMTP_HOST} = {OPTION_VALUE_SMTP_HOST} + """ + ), + ), + ( + SECTION_SMTP, + OPTION_KEY_SMTP_MAIL_FROM, + HEADERS_JSON, + 200, + { + "sections": [ + { + "name": SECTION_SMTP, + "options": [ + {"key": OPTION_KEY_SMTP_MAIL_FROM, "value": OPTION_VALUE_SMTP_MAIL_FROM}, + ], + }, + ], + }, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_JSON, + 200, + { + "sections": [ + { + "name": SECTION_DATABASE, + "options": [ + {"key": OPTION_KEY_SQL_ALCHEMY_CONN, "value": OPTION_VALUE_SENSITIVE_HIDDEN}, + ], + }, + ], + }, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_DATABASE}] + {OPTION_KEY_SQL_ALCHEMY_CONN} = {OPTION_VALUE_SENSITIVE_HIDDEN} + """ + ), + ), + (SECTION_CORE, OPTION_KEY_PARALLELISM, HEADERS_JSON, 403, FORBIDDEN_RESPONSE), + ( + SECTION_NOT_EXIST, + OPTION_KEY_PARALLELISM, + HEADERS_JSON, + 404, + {"detail": f"Option [{SECTION_NOT_EXIST}/{OPTION_KEY_PARALLELISM}] not found."}, + ), + ( + SECTION_CORE, + OPTION_NOT_EXIST, + HEADERS_JSON, + 404, + {"detail": f"Option [{SECTION_CORE}/{OPTION_NOT_EXIST}] not found."}, + ), + ], + ) + def test_get_config_value( + self, test_client, section, option, headers, expected_status_code, expected_response + ): + if expected_status_code == 403: + with conf_vars(AIRFLOW_CONFIG_DISABLE_EXPOSE_CONFIG): + response = test_client.get( + f"/public/config/section/{section}/option/{option}", headers=headers + ) + else: + response = test_client.get(f"/public/config/section/{section}/option/{option}", headers=headers) + self._validate_response(headers, expected_response, expected_status_code, response) + + @pytest.mark.parametrize( + "section, option, headers, expected_status_code, expected_response", + [ + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_JSON, + 200, + GET_CONFIG_VALUE_NON_SENSITIVE_ONLY_DATABASE_SQL_ALCHEMY_CONN_JSON_RESPONSE, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_JSON_UTF8, + 200, + GET_CONFIG_VALUE_NON_SENSITIVE_ONLY_DATABASE_SQL_ALCHEMY_CONN_JSON_RESPONSE, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_ANY, + 200, + GET_CONFIG_VALUE_NON_SENSITIVE_ONLY_DATABASE_SQL_ALCHEMY_CONN_JSON_RESPONSE, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_NONE, + 200, + GET_CONFIG_VALUE_NON_SENSITIVE_ONLY_DATABASE_SQL_ALCHEMY_CONN_JSON_RESPONSE, + ), + ( + SECTION_DATABASE, + OPTION_KEY_SQL_ALCHEMY_CONN, + HEADERS_TEXT, + 200, + textwrap.dedent( + f"""\ + [{SECTION_DATABASE}] + {OPTION_KEY_SQL_ALCHEMY_CONN} = {OPTION_VALUE_SENSITIVE_HIDDEN} + """ + ), + ), + ], + ) + def test_get_config_value_non_sensitive_only( + self, test_client, section, option, headers, expected_status_code, expected_response + ): + with conf_vars(AIRFLOW_CONFIG_NON_SENSITIVE_ONLY_CONFIG): + response = test_client.get(f"/public/config/section/{section}/option/{option}", headers=headers) + self._validate_response(headers, expected_response, expected_status_code, response) diff --git a/tests/api_fastapi/core_api/routes/public/test_dag_sources.py b/tests/api_fastapi/core_api/routes/public/test_dag_sources.py index d8f81482cfbff..6c040d57f9cbb 100644 --- a/tests/api_fastapi/core_api/routes/public/test_dag_sources.py +++ b/tests/api_fastapi/core_api/routes/public/test_dag_sources.py @@ -74,11 +74,14 @@ def test_should_respond_200_text(self, test_client, test_dag): json.loads(response.content.decode()) assert response.headers["Content-Type"].startswith("text/plain") - def test_should_respond_200_json(self, test_client, test_dag): + @pytest.mark.parametrize( + "headers", [{"Accept": "application/json"}, {"Accept": "application/json; charset=utf-8"}, {}] + ) + def test_should_respond_200_json(self, test_client, test_dag, headers): dag_content = self._get_dag_file_code(test_dag.fileloc) response: Response = test_client.get( f"{API_PREFIX}/{TEST_DAG_ID}", - headers={"Accept": "application/json"}, + headers=headers, ) assert isinstance(response, Response) assert 200 == response.status_code