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
25 changes: 25 additions & 0 deletions airflow-core/newsfragments/49017.significant.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
Renamed FAB related configuration.

* Rename configuration ``webserver.config_file`` as ``fab.config_file``
* Rename configuration ``webserver.session_backend`` as ``fab.session_backend``
* Rename configuration ``webserver.base_url`` as ``api.base_url``

* Types of change

* [ ] Dag changes
* [x] Config changes
* [ ] API changes
* [ ] CLI changes
* [ ] Behaviour changes
* [ ] Plugin changes
* [ ] Dependency changes
* [ ] Code interface changes

* Migration rules needed


* ``airflow config lint``

* [x] ``webserver.config_file`` → ``fab.config_file``
* [x] ``webserver.session_backend`` → ``fab.session_backend``
* [x] ``webserver.base_url`` → ``api.base_url``
3 changes: 0 additions & 3 deletions airflow-core/src/airflow/__main__.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
# any possible import cycles with settings downstream.
from airflow import configuration
from airflow.cli import cli_parser
from airflow.configuration import write_webserver_configuration_if_needed


def main():
Expand All @@ -53,8 +52,6 @@ def main():
from airflow.configuration import write_default_airflow_configuration_if_needed

conf = write_default_airflow_configuration_if_needed()
if args.subcommand in ["webserver", "internal-api", "worker"]:
write_webserver_configuration_if_needed(conf)
args.func(args)


Expand Down
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 @@ -36,6 +36,7 @@
from airflow.api_fastapi.execution_api.app import create_task_execution_api_app
from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException
from airflow.utils.providers_configuration_loader import providers_configuration_loaded

if TYPE_CHECKING:
from airflow.api_fastapi.auth.managers.base_auth_manager import BaseAuthManager
Expand Down Expand Up @@ -67,6 +68,7 @@ async def lifespan(app: FastAPI):
yield


@providers_configuration_loaded
def create_app(apps: str = "all") -> FastAPI:
apps_list = apps.split(",") if apps else ["all"]

Expand Down
12 changes: 12 additions & 0 deletions airflow-core/src/airflow/cli/commands/config_command.py
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,14 @@ def message(self) -> str | None:
config=ConfigParameter("webserver", option="auth_rate_limit"),
renamed_to=ConfigParameter("fab", "auth_rate_limit"),
),
ConfigChange(
config=ConfigParameter("webserver", "config_file"),
renamed_to=ConfigParameter("fab", "config_file"),
),
ConfigChange(
config=ConfigParameter("webserver", "session_backend"),
renamed_to=ConfigParameter("fab", "session_backend"),
),
ConfigChange(
config=ConfigParameter("webserver", "session_lifetime_days"),
renamed_to=ConfigParameter("fab", "session_lifetime_minutes"),
Expand All @@ -339,6 +347,10 @@ def message(self) -> str | None:
config=ConfigParameter("webserver", "session_lifetime_minutes"),
renamed_to=ConfigParameter("fab", "session_lifetime_minutes"),
),
ConfigChange(
config=ConfigParameter("webserver", "base_url"),
renamed_to=ConfigParameter("api", "base_url"),
),
ConfigChange(
config=ConfigParameter("webserver", "web_server_host"),
renamed_to=ConfigParameter("api", "host"),
Expand Down
37 changes: 1 addition & 36 deletions airflow-core/src/airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1708,22 +1708,6 @@ webserver:
type: string
example: ~
default: "Access is Denied"
config_file:
description: |
Path of webserver config file used for configuring the webserver parameters
version_added: 2.7.0
type: string
example: ~
default: "{AIRFLOW_HOME}/webserver_config.py"
base_url:
description: |
The base url of your website: Airflow cannot guess what domain or CNAME you are using.
This is used to create links in the Log Url column in the Browse - Task Instances menu,
as well as in any automated emails sent by Airflow that contain links to your webserver.
version_added: ~
type: string
example: ~
default: "http://localhost:8080"
default_ui_timezone:
description: |
Default timezone to display all dates in the UI, can be UTC, system, or
Expand All @@ -1734,28 +1718,9 @@ webserver:
example: "America/New_York"
# Default is left as UTC for now so the date's don't "suddenly" change on upgrade
default: "UTC"
session_backend:
description: |
The type of backend used to store web session data, can be ``database`` or ``securecookie``. For the
``database`` backend, sessions are store in the database and they can be
managed there (for example when you reset password of the user, all sessions for that user are
deleted). For the ``securecookie`` backend, sessions are stored in encrypted cookies on the client
side. The ``securecookie`` mechanism is 'lighter' than database backend, but sessions are not deleted
when you reset password of the user, which means that other than waiting for expiry time, the only
way to invalidate all sessions for a user is to change secret_key and restart webserver (which
also invalidates and logs out all other user's sessions).

When you are using ``database`` backend, make sure to keep your database session table small
by periodically running ``airflow db clean --table session`` command, especially if you have
automated API calls that will create a new session for each call rather than reuse the sessions
stored in browser cookies.
version_added: 2.2.4
type: string
example: securecookie
default: database
secret_key:
description: |
Secret key used to run your flask app. It should be as random as possible. However, when running
Secret key used to run your api server. It should be as random as possible. However, when running
more than 1 instances of webserver, make sure all of them use the same ``secret_key`` otherwise
one of them will error with "CSRF session token is missing".
The webserver key is also used to authorize requests to Celery workers when logs are retrieved.
Expand Down
16 changes: 0 additions & 16 deletions airflow-core/src/airflow/configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@
from airflow.secrets import DEFAULT_SECRETS_SEARCH_PATH
from airflow.utils import yaml
from airflow.utils.module_loading import import_string
from airflow.utils.providers_configuration_loader import providers_configuration_loaded
from airflow.utils.weight_rule import WeightRule

if TYPE_CHECKING:
Expand Down Expand Up @@ -2108,23 +2107,9 @@ def initialize_config() -> AirflowConfigParser:
# file on top of it.
if airflow_config_parser.getboolean("core", "unit_test_mode"):
airflow_config_parser.load_test_config()
# Set the WEBSERVER_CONFIG variable
global WEBSERVER_CONFIG
WEBSERVER_CONFIG = airflow_config_parser.get("webserver", "config_file")
return airflow_config_parser


@providers_configuration_loaded
def write_webserver_configuration_if_needed(airflow_config_parser: AirflowConfigParser):
webserver_config = airflow_config_parser.get("webserver", "config_file")
if not os.path.isfile(webserver_config):
import shutil

pathlib.Path(webserver_config).parent.mkdir(parents=True, exist_ok=True)
log.info("Creating new FAB webserver config file in: %s", webserver_config)
shutil.copy(_default_config_file_path("default_webserver_config.py"), webserver_config)


def make_group_other_inaccessible(file_path: str):
try:
permissions = os.stat(file_path)
Expand Down Expand Up @@ -2264,7 +2249,6 @@ def initialize_auth_manager() -> BaseAuthManager:
SECRET_KEY = b64encode(os.urandom(16)).decode("utf-8")
FERNET_KEY = "" # Set only if needed when generating a new file
JWT_SECRET_KEY = ""
WEBSERVER_CONFIG = "" # Set by initialize_config

conf: AirflowConfigParser = initialize_config()
secrets_backend_list = initialize_secrets_backends()
Expand Down
4 changes: 2 additions & 2 deletions airflow-core/src/airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -1915,10 +1915,10 @@ def generate_command(
def log_url(self) -> str:
"""Log URL for TaskInstance."""
run_id = quote(self.run_id)
base_url = conf.get_mandatory_value("webserver", "BASE_URL")
base_url = conf.get_mandatory_value("api", "BASE_URL")
map_index = f"/mapped/{self.map_index}" if self.map_index >= 0 else ""
try_number = f"?try_number={self.try_number}" if self.try_number > 0 else ""
_log_uri = f"{base_url}/dags/{self.dag_id}/runs/{run_id}/tasks/{self.task_id}{map_index}{try_number}"
_log_uri = f"{base_url}dags/{self.dag_id}/runs/{run_id}/tasks/{self.task_id}{map_index}{try_number}"

return _log_uri

Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/settings.py
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
from sqlalchemy.pool import NullPool

from airflow import __version__ as airflow_version, policies
from airflow.configuration import AIRFLOW_HOME, WEBSERVER_CONFIG, conf # noqa: F401
from airflow.configuration import AIRFLOW_HOME, conf
from airflow.exceptions import AirflowInternalRuntimeError
from airflow.logging_config import configure_logging
from airflow.utils.orm_event_handlers import setup_event_handlers
Expand Down
7 changes: 6 additions & 1 deletion airflow-core/src/airflow/utils/db_cleanup.py
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,12 @@ def readable_config(self):
_TableConfig(table_name="deadline", recency_column_name="deadline"),
]

if conf.get("webserver", "session_backend") == "database":
# We need to have `fallback="database"` because this is executed at top level code and provider configuration
# might not be loaded
if (
"FabAuthManager" in conf.get("core", "auth_manager")
and conf.get("fab", "session_backend", fallback="database") == "database"
):
config_list.append(_TableConfig(table_name="session", recency_column_name="expiry"))

config_dict: dict[str, _TableConfig] = {x.orm_model.name: x for x in sorted(config_list)}
Expand Down
28 changes: 27 additions & 1 deletion providers/fab/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -79,10 +79,36 @@ config:
auth_backends:
description: |
Comma separated list of auth backends to authenticate users of the API.
version_added: 2.3.0
version_added: 2.0.0
type: string
example: ~
default: "airflow.providers.fab.auth_manager.api.auth.backend.session"
config_file:
description: |
Path of webserver config file used for configuring the webserver parameters
version_added: 2.0.0
type: string
example: ~
default: "{AIRFLOW_HOME}/webserver_config.py"
session_backend:
description: |
The type of backend used to store web session data, can be ``database`` or ``securecookie``. For the
``database`` backend, sessions are store in the database and they can be
managed there (for example when you reset password of the user, all sessions for that user are
deleted). For the ``securecookie`` backend, sessions are stored in encrypted cookies on the client
side. The ``securecookie`` mechanism is 'lighter' than database backend, but sessions are not
deleted when you reset password of the user, which means that other than waiting for expiry time,
the only way to invalidate all sessions for a user is to change secret_key and restart webserver
(which also invalidates and logs out all other user's sessions).

When you are using ``database`` backend, make sure to keep your database session table small
by periodically running ``airflow db clean --table session`` command, especially if you have
automated API calls that will create a new session for each call rather than reuse the sessions
stored in browser cookies.
version_added: 2.0.0
type: string
example: "securecookie"
default: "database"
session_lifetime_minutes:
description: |
The UI cookie lifetime in minutes. User will be logged out from UI after
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ def _return_appbuilder(app: Flask) -> AirflowAppBuilder:
def get_application_builder() -> Generator[AirflowAppBuilder, None, None]:
static_folder = os.path.join(os.path.dirname(airflow.__file__), "www", "static")
flask_app = Flask(__name__, static_folder=static_folder)
webserver_config = conf.get_mandatory_value("webserver", "config_file")
webserver_config = conf.get_mandatory_value("fab", "config_file")
with flask_app.app_context():
# Enable customizations in webserver_config.py to be applied via Flask.current_app.
flask_app.config.from_pyfile(webserver_config, silent=True)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2079,7 +2079,7 @@ def _rotate_session_id(self):
We need to do this upon successful authentication when using the
database session backend.
"""
if conf.get("webserver", "SESSION_BACKEND") == "database":
if conf.get("fab", "SESSION_BACKEND") == "database":
session.sid = str(uuid.uuid4())

def _get_microsoft_jwks(self) -> list[dict[str, Any]]:
Expand Down
16 changes: 15 additions & 1 deletion providers/fab/src/airflow/providers/fab/get_provider_info.py
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,25 @@ def get_provider_info():
},
"auth_backends": {
"description": "Comma separated list of auth backends to authenticate users of the API.\n",
"version_added": "2.3.0",
"version_added": "2.0.0",
"type": "string",
"example": None,
"default": "airflow.providers.fab.auth_manager.api.auth.backend.session",
},
"config_file": {
"description": "Path of webserver config file used for configuring the webserver parameters\n",
"version_added": "2.0.0",
"type": "string",
"example": None,
"default": "{AIRFLOW_HOME}/webserver_config.py",
},
"session_backend": {
"description": "The type of backend used to store web session data, can be ``database`` or ``securecookie``. For the\n``database`` backend, sessions are store in the database and they can be\nmanaged there (for example when you reset password of the user, all sessions for that user are\ndeleted). For the ``securecookie`` backend, sessions are stored in encrypted cookies on the client\nside. The ``securecookie`` mechanism is 'lighter' than database backend, but sessions are not\ndeleted when you reset password of the user, which means that other than waiting for expiry time,\nthe only way to invalidate all sessions for a user is to change secret_key and restart webserver\n(which also invalidates and logs out all other user's sessions).\n\nWhen you are using ``database`` backend, make sure to keep your database session table small\nby periodically running ``airflow db clean --table session`` command, especially if you have\nautomated API calls that will create a new session for each call rather than reuse the sessions\nstored in browser cookies.\n",
"version_added": "2.0.0",
"type": "string",
"example": "securecookie",
"default": "database",
},
"session_lifetime_minutes": {
"description": "The UI cookie lifetime in minutes. User will be logged out from UI after\n``[fab] session_lifetime_minutes`` of non-activity\n",
"version_added": "2.0.0",
Expand Down
2 changes: 1 addition & 1 deletion providers/fab/src/airflow/providers/fab/www/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ def create_app(enable_plugins: bool):
flask_app.config["SQLALCHEMY_TRACK_MODIFICATIONS"] = False
flask_app.config["PERMANENT_SESSION_LIFETIME"] = timedelta(minutes=get_session_lifetime_config())

webserver_config = conf.get_mandatory_value("webserver", "config_file")
webserver_config = conf.get_mandatory_value("fab", "config_file")
# Enable customizations in webserver_config.py to be applied via Flask.current_app.
with flask_app.app_context():
flask_app.config.from_pyfile(webserver_config, silent=True)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
def init_airflow_session_interface(app):
"""Set airflow session interface."""
config = app.config.copy()
selected_backend = conf.get("webserver", "SESSION_BACKEND")
selected_backend = conf.get("fab", "SESSION_BACKEND")
# A bit of a misnomer - normally cookies expire whenever the browser is closed
# or when they hit their expiry datetime, whichever comes first. "Permanent"
# cookies only expire when they hit their expiry datetime, and can outlive
Expand Down Expand Up @@ -59,6 +59,6 @@ def make_session_permanent():
else:
raise AirflowConfigException(
"Unrecognized session backend specified in "
f"web_server_session_backend: '{selected_backend}'. Please set "
f"[fab] session_backend: '{selected_backend}'. Please set "
"this to either 'database' or 'securecookie'."
)
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
def flask_app():
"""Fixture to set up the Flask app with the necessary configuration."""
# Get the webserver config file path
webserver_config = conf.get_mandatory_value("webserver", "config_file")
webserver_config = conf.get_mandatory_value("fab", "config_file")

with get_application_builder() as appbuilder:
flask_app = appbuilder.app
Expand Down