From d8fdac96135a8921c4aeaa5fc4405b9e7b23bc49 Mon Sep 17 00:00:00 2001 From: Gayathri Srividya Rajavarapu Date: Tue, 2 Jun 2026 00:59:36 +0530 Subject: [PATCH] Fix AttributeError when notifier renders templates with SerializedDAG MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When 'airflow dags test' triggers a DAG callback that uses a notifier, the context's 'dag' key holds a SerializedDAG instance which does not have a get_template_env method. BaseNotifier.render_template_fields passes this dag to Templater.get_template_env, which unconditionally calls dag.get_template_env() — raising AttributeError. Guard the call with hasattr() so that a dag lacking get_template_env (such as SerializedDAG) silently falls back to SandboxedEnvironment. closes: #64649 --- .../sdk/definitions/_internal/templater.py | 2 +- .../tests/task_sdk/bases/test_notifier.py | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/task-sdk/src/airflow/sdk/definitions/_internal/templater.py b/task-sdk/src/airflow/sdk/definitions/_internal/templater.py index cfe4a6100e482..393f4dfb24122 100644 --- a/task-sdk/src/airflow/sdk/definitions/_internal/templater.py +++ b/task-sdk/src/airflow/sdk/definitions/_internal/templater.py @@ -76,7 +76,7 @@ def get_template_env(self, dag: DAG | None = None) -> jinja2.Environment: # for most of the functionalities. It is imported by get_template_env() # though, so we don't need to put this after the 'if dag' check. - if dag: + if dag and hasattr(dag, "get_template_env"): return dag.get_template_env(force_sandboxed=False) return SandboxedEnvironment(cache_size=0) diff --git a/task-sdk/tests/task_sdk/bases/test_notifier.py b/task-sdk/tests/task_sdk/bases/test_notifier.py index b8cedaa518831..40dc411af020f 100644 --- a/task-sdk/tests/task_sdk/bases/test_notifier.py +++ b/task-sdk/tests/task_sdk/bases/test_notifier.py @@ -97,3 +97,23 @@ def test_notifier_call_with_prepared_context(self, caplog): } ) assert notifier.message == "task: some_task" + + def test_render_template_fields_with_serialized_dag(self): + """Rendering template fields with a SerializedDAG-like object should not raise AttributeError. + + When `airflow dags test` runs callbacks, the context contains a SerializedDAG which does + not have a `get_template_env` method. The notifier must fall back to a SandboxedEnvironment + rather than raising AttributeError. + """ + + class FakeSerializedDAG: + """Simulate SerializedDAG which lacks get_template_env.""" + + dag_id = "test_dag" + render_template_as_native_obj = False + + notifier = MockNotifier(message="Hello {{ dag.dag_id }}") + context: Context = {"dag": FakeSerializedDAG()} + # Must not raise AttributeError: 'FakeSerializedDAG' object has no attribute 'get_template_env' + notifier.render_template_fields(context) + assert notifier.message == "Hello test_dag"