diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 15ac435ff7d7e..bf324a297dd38 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1010,7 +1010,6 @@ def get_triggering_events() -> dict[str, list[AssetEvent]]: # * KNOWN_CONTEXT_KEYS in airflow/utils/context.py # * Table in docs/apache-airflow/templates-ref.rst context: dict[str, Any] = { - "conf": conf, "dag": dag, "dag_run": dag_run, "data_interval_end": timezone.coerce_datetime(data_interval.end), diff --git a/airflow/utils/context.py b/airflow/utils/context.py index 67f31a3f2b6b4..c6cf2db498532 100644 --- a/airflow/utils/context.py +++ b/airflow/utils/context.py @@ -67,7 +67,6 @@ # * Context in airflow/utils/context.pyi. # * Table in docs/apache-airflow/templates-ref.rst KNOWN_CONTEXT_KEYS: set[str] = { - "conf", "conn", "dag", "dag_run", diff --git a/airflow/utils/context.pyi b/airflow/utils/context.pyi index b2fdc98a7459a..4696e48c75d6f 100644 --- a/airflow/utils/context.pyi +++ b/airflow/utils/context.pyi @@ -32,7 +32,6 @@ from typing import Any, overload from pendulum import DateTime from sqlalchemy.orm import Session -from airflow.configuration import AirflowConfigParser from airflow.models.asset import AssetEvent from airflow.models.baseoperator import BaseOperator from airflow.models.dag import DAG @@ -100,7 +99,6 @@ class InletEventsAccessors(Mapping[Asset | AssetAlias, InletEventsAccessor]): # * KNOWN_CONTEXT_KEYS in airflow/utils/context.py # * Table in docs/apache-airflow/templates-ref.rst class Context(TypedDict, total=False): - conf: AirflowConfigParser conn: Any dag: DAG dag_run: DagRun | DagRunPydantic diff --git a/docs/apache-airflow/templates-ref.rst b/docs/apache-airflow/templates-ref.rst index 78e5c17f9925b..cf7b015141b94 100644 --- a/docs/apache-airflow/templates-ref.rst +++ b/docs/apache-airflow/templates-ref.rst @@ -79,8 +79,6 @@ Variable Type Description ``{{ conn }}`` Airflow connections. See `Airflow Connections in Templates`_ below. ``{{ task_instance_key_str }}`` str | A unique, human-readable key to the task instance. The format is | ``{dag_id}__{task_id}__{ds_nodash}``. -``{{ conf }}`` AirflowConfigParser | The full configuration object representing the content of your - | ``airflow.cfg``. See :mod:`airflow.configuration.conf`. ``{{ run_id }}`` str The currently running :class:`~airflow.models.dagrun.DagRun` run ID. ``{{ dag_run }}`` DagRun The currently running :class:`~airflow.models.dagrun.DagRun`. ``{{ test_mode }}`` bool Whether the task instance was run by the ``airflow test`` CLI. diff --git a/newsfragments/44820.significant.rst b/newsfragments/44820.significant.rst new file mode 100644 index 0000000000000..61b7c968ff333 --- /dev/null +++ b/newsfragments/44820.significant.rst @@ -0,0 +1,38 @@ +Removed ``conf`` from the Task template context + +The ``conf`` variable, which provided access to the full Airflow configuration (``airflow.cfg``), has been +removed from the Task (Jinja2) template context for security and simplicity. If you +need specific configuration values in your tasks, retrieve them explicitly in your DAG or task code +using the ``airflow.configuration.conf`` module. + +For users retrieving the webserver URL (e.g., to include log links in task or callbacks), one of the +most common use-case, use the ``ti.log_url`` property available in the ``TaskInstance`` context instead. + +Example: + +.. code-block:: python + + PythonOperator( + task_id="my_task", + python_callable=my_task_callable, + on_failure_callback=SmtpNotifier( + from_email="example@example.com", + to="example@example.com", + subject="Task {{ ti.task_id }} failed", + html_content="Task {{ ti.task_id }} failed. Log URL: {{ ti.log_url }}", + ), + ) + +* Types of change + + * [x] DAG changes + * [ ] Config changes + * [ ] API changes + * [ ] CLI changes + * [ ] Behaviour changes + * [ ] Plugin changes + * [ ] Dependency changes + +* Migration rules needed + + * Remove context key ``conf``