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
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ Most task handlers send logs upon completion of a task. In order to view logs in
In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``.

The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer. Defaults are 8793 and 8794, respectively.
Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems.
Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option in ``[api]`` section. You must ensure that the key matches so that communication can take place without problems.

We are using `Gunicorn <https://gunicorn.org/>`__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings <https://docs.gunicorn.org/en/latest/settings.html#settings>`__.

Expand Down
2 changes: 1 addition & 1 deletion airflow-core/docs/cli-and-env-variables-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ Environment Variables
* ``result_backend`` in ``[celery]`` section
* ``password`` in ``[atlas]`` section
* ``smtp_password`` in ``[smtp]`` section
* ``secret_key`` in ``[webserver]`` section
* ``secret_key`` in ``[api]`` section

.. envvar:: AIRFLOW__{SECTION}__{KEY}_SECRET

Expand Down
2 changes: 1 addition & 1 deletion airflow-core/docs/configurations-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ can set in ``airflow.cfg`` file or using environment variables.

Use the same configuration across all the Airflow components. While each component
does not require all, some configurations need to be same otherwise they would not
work as expected. A good example for that is :ref:`secret_key<config:webserver__secret_key>` which
work as expected. A good example for that is :ref:`secret_key<config:api__secret_key>` which
should be same on the Webserver and Worker to allow Webserver to fetch logs from Worker.

The webserver key is also used to authorize requests to Celery workers when logs are retrieved. The token
Expand Down
4 changes: 2 additions & 2 deletions airflow-core/docs/howto/set-config.rst
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ The following config options support this ``_cmd`` and ``_secret`` version:
* ``result_backend`` in ``[celery]`` section
* ``password`` in ``[atlas]`` section
* ``smtp_password`` in ``[smtp]`` section
* ``secret_key`` in ``[webserver]`` section
* ``secret_key`` in ``[api]`` section

The ``_cmd`` config options can also be set using a corresponding environment variable
the same way the usual config options can. For example:
Expand Down Expand Up @@ -159,7 +159,7 @@ the example below.
.. note::
Use the same configuration across all the Airflow components. While each component
does not require all, some configurations need to be same otherwise they would not
work as expected. A good example for that is :ref:`secret_key<config:webserver__secret_key>` which
work as expected. A good example for that is :ref:`secret_key<config:api__secret_key>` which
should be same on the Webserver and Worker to allow Webserver to fetch logs from Worker.

The webserver key is also used to authorize requests to Celery workers when logs are retrieved. The token
Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/api_fastapi/core_api/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ def init_config(app: FastAPI) -> None:
# and 9 (slowest, most compression)
app.add_middleware(GZipMiddleware, minimum_size=1024, compresslevel=5)

app.state.secret_key = get_signing_key("webserver", "secret_key")
app.state.secret_key = get_signing_key("api", "secret_key")


def init_error_handlers(app: FastAPI) -> None:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ def get_timetable_summary(cls, tts: str | None) -> str | None:
@property
def file_token(self) -> str:
"""Return file token."""
serializer = URLSafeSerializer(conf.get_mandatory_value("webserver", "secret_key"))
serializer = URLSafeSerializer(conf.get_mandatory_value("api", "secret_key"))
payload = {
"bundle_name": self.bundle_name,
"relative_fileloc": self.relative_fileloc,
Expand Down
4 changes: 4 additions & 0 deletions airflow-core/src/airflow/cli/commands/config_command.py
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,10 @@ def message(self) -> str | None:
config=ConfigParameter("webserver", "base_url"),
renamed_to=ConfigParameter("api", "base_url"),
),
ConfigChange(
config=ConfigParameter("webserver", "secret_key"),
renamed_to=ConfigParameter("api", "secret_key"),
),
ConfigChange(
config=ConfigParameter("webserver", "web_server_host"),
renamed_to=ConfigParameter("api", "host"),
Expand Down
28 changes: 14 additions & 14 deletions airflow-core/src/airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1318,6 +1318,20 @@ secrets:
api:
description: ~
options:
secret_key:
description: |
Secret key used to run your api server. It should be as random as possible. However, when running
more than 1 instances of the api, make sure all of them use the same ``secret_key`` otherwise
one of them will error with "CSRF session token is missing".
The api key is also used to authorize requests to Celery workers when logs are retrieved.
The token generated using the secret key has a short expiry time though - make sure that time on
ALL the machines that you run airflow components on is synchronized (for example using ntpd)
otherwise you might get "forbidden" errors when the logs are accessed.
version_added: ~
type: string
sensitive: true
example: ~
default: "{SECRET_KEY}"
expose_config:
description: |
Expose the configuration file in the web server. Set to ``non-sensitive-only`` to show all values
Expand Down Expand Up @@ -1727,20 +1741,6 @@ operators:
webserver:
description: ~
options:
secret_key:
description: |
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.
The token generated using the secret key has a short expiry time though - make sure that time on
ALL the machines that you run airflow components on is synchronized (for example using ntpd)
otherwise you might get "forbidden" errors when the logs are accessed.
version_added: ~
type: string
sensitive: true
example: ~
default: "{SECRET_KEY}"
grid_view_sorting_order:
description: |
Sorting order in grid view. Valid values are: ``topological``, ``hierarchical_alphabetical``
Expand Down
1 change: 1 addition & 0 deletions airflow-core/src/airflow/configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,7 @@ def sensitive_config_values(self) -> set[tuple[str, str]]:
("fab", "navbar_text_color"): ("webserver", "navbar_text_color", "3.0.2"),
("fab", "navbar_hover_color"): ("webserver", "navbar_hover_color", "3.0.2"),
("fab", "navbar_text_hover_color"): ("webserver", "navbar_text_hover_color", "3.0.2"),
("api", "secret_key"): ("webserver", "secret_key", "3.0.2"),
}

# A mapping of new section -> (old section, since_version).
Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/utils/log/file_task_handler.py
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ def _fetch_logs_from_service(url, log_relative_path):

timeout = conf.getint("webserver", "log_fetch_timeout_sec", fallback=None)
generator = JWTGenerator(
secret_key=get_signing_key("webserver", "secret_key"),
secret_key=get_signing_key("api", "secret_key"),
# Since we are using a secret key, we need to be explicit about the algorithm here too
algorithm="HS512",
private_key=None,
Expand Down
2 changes: 1 addition & 1 deletion airflow-core/src/airflow/utils/serve_logs.py
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ def create_app():
raise ImportError(f"Unable to load {log_config_class} due to error: {e}")
signer = JWTValidator(
issuer=None,
secret_key=get_signing_key("webserver", "secret_key"),
secret_key=get_signing_key("api", "secret_key"),
algorithm="HS512",
leeway=leeway,
audience="task-instance-logs",
Expand Down
13 changes: 8 additions & 5 deletions airflow-core/tests/unit/core/test_configuration.py
Original file line number Diff line number Diff line change
Expand Up @@ -609,8 +609,8 @@ def test_get_section_should_respect_cmd_env_variable(self, tmp_path, monkeypatch
cmd_file.write_text("#!/usr/bin/env bash\necho -n difficult_unpredictable_cat_password\n")
cmd_file.chmod(0o0555)

monkeypatch.setenv("AIRFLOW__WEBSERVER__SECRET_KEY_CMD", str(cmd_file))
content = conf.getsection("webserver")
monkeypatch.setenv("AIRFLOW__API__SECRET_KEY_CMD", str(cmd_file))
content = conf.getsection("api")
assert content["secret_key"] == "difficult_unpredictable_cat_password"

@pytest.mark.parametrize(
Expand All @@ -628,11 +628,13 @@ def test_config_value_types(self, key, type):
assert isinstance(section_dict[key], type)

def test_command_from_env(self):
test_cmdenv_config = textwrap.dedent("""\
test_cmdenv_config = textwrap.dedent(
"""\
[testcmdenv]
itsacommand=NOT OK
notacommand=OK
""")
"""
)
test_cmdenv_conf = AirflowConfigParser()
test_cmdenv_conf.read_string(test_cmdenv_config)
test_cmdenv_conf.sensitive_config_values.add(("testcmdenv", "itsacommand"))
Expand Down Expand Up @@ -1713,7 +1715,7 @@ def test_sensitive_values():
("core", "fernet_key"),
("core", "internal_api_secret_key"),
("api_auth", "jwt_secret"),
("webserver", "secret_key"),
("api", "secret_key"),
("secrets", "backend_kwargs"),
("sentry", "sentry_dsn"),
("database", "sql_alchemy_engine_args"),
Expand All @@ -1724,6 +1726,7 @@ def test_sensitive_values():
("celery", "result_backend"),
("opensearch", "username"),
("opensearch", "password"),
("webserver", "secret_key"),
}
all_keys = {(s, k) for s, v in conf.configuration_description.items() for k in v.get("options")}
suspected_sensitive = {(s, k) for (s, k) in all_keys if k.endswith(("password", "kwargs"))}
Expand Down
4 changes: 2 additions & 2 deletions contributing-docs/testing/k8s_tests.rst
Original file line number Diff line number Diff line change
Expand Up @@ -469,8 +469,8 @@ Should show the status of current KinD cluster.
unnecessary restarts of your Airflow components.

Information on how to set a static webserver secret key can be found here:
https://airflow.apache.org/docs/helm-chart/stable/production-guide.html#webserver-secret-key
Deployed kind-airflow-python-3.9-v1.24.2 with airflow Helm Chart.
https://airflow.apache.org/docs/helm-chart/stable/production-guide.html#api-secret-key
Deployed kind-airflow-python-3.9-v1.24.2 with Airflow Helm Chart.

Airflow for Python 3.9 and K8S version v1.24.2 has been successfully deployed.

Expand Down
2 changes: 1 addition & 1 deletion dev/breeze/src/airflow_breeze/params/shell_params.py
Original file line number Diff line number Diff line change
Expand Up @@ -527,7 +527,7 @@ def env_variables_for_docker_commands(self) -> dict[str, str]:
"AIRFLOW__CORE__SIMPLE_AUTH_MANAGER_PASSWORDS_FILE",
"/opt/airflow/dev/breeze/src/airflow_breeze/files/simple_auth_manager_passwords.json",
)
_set_var(_env, "AIRFLOW__WEBSERVER__SECRET_KEY", b64encode(os.urandom(16)).decode("utf-8"))
_set_var(_env, "AIRFLOW__API__SECRET_KEY", b64encode(os.urandom(16)).decode("utf-8"))
if self.executor == EDGE_EXECUTOR:
_set_var(
_env,
Expand Down
4 changes: 2 additions & 2 deletions devel-common/src/tests_common/pytest_plugin.py
Original file line number Diff line number Diff line change
Expand Up @@ -1757,10 +1757,10 @@ def secret_key() -> str:
"""Return secret key configured."""
from airflow.configuration import conf

the_key = conf.get("webserver", "SECRET_KEY")
the_key = conf.get("api", "SECRET_KEY")
if the_key is None:
raise RuntimeError(
"The secret key SHOULD be configured as `[webserver] secret_key` in the "
"The secret key SHOULD be configured as `[api] secret_key` in the "
"configuration/environment at this stage! "
)
return the_key
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 @@ -55,7 +55,7 @@ def create_app(enable_plugins: bool):
from airflow.providers.fab.auth_manager.fab_auth_manager import FabAuthManager

flask_app = Flask(__name__)
flask_app.secret_key = conf.get("webserver", "SECRET_KEY")
flask_app.secret_key = conf.get("api", "SECRET_KEY")
flask_app.config["SQLALCHEMY_DATABASE_URI"] = conf.get("database", "SQL_ALCHEMY_CONN")
flask_app.config["SQLALCHEMY_TRACK_MODIFICATIONS"] = False
flask_app.config["PERMANENT_SESSION_LIFETIME"] = timedelta(minutes=get_session_lifetime_config())
Expand Down