Skip to content
Open
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 @@ -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)

Expand Down
20 changes: 20 additions & 0 deletions task-sdk/tests/task_sdk/bases/test_notifier.py
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Loading