I agree that both of these proposals are "best practices" that should be
encouraged but I'm not clear on what we're voting for here.
________________________________
From: Wei Lee <[email protected]>
Sent: Tuesday, March 17, 2026 4:14 AM
To: [email protected] <[email protected]>
Subject: RE: [EXT] [VOTE] New DAG Authorship Best Practice: Use `.output`
Instead of `xcom_pull` Template Strings
CAUTION: This email originated from outside of the organization. Do not click
links or open attachments unless you can confirm the sender and know the
content is safe.
AVERTISSEMENT: Ce courrier électronique provient d’un expéditeur externe. Ne
cliquez sur aucun lien et n’ouvrez aucune pièce jointe si vous ne pouvez pas
confirmer l’identité de l’expéditeur et si vous n’êtes pas certain que le
contenu ne présente aucun risque.
Thanks Illya for initiating this!
In case others also find it not easy to read, I tried to fix the format a bit.
+1 for this proposal.
Best,
Wei
---
Hello everyone,
I'd like to initiate a vote on another new DAG authorship best practice.
Proposal:
When passing the output of one task to another via a Jinja template string
containing a single `xcom_pull` call — such as "{{
ti.xcom_pull(task_ids='some_task') }}" — the `.output` attribute on the task
object should be preferred instead.
Before:
task_1 = PythonOperator(task_id="task_1", python_callable=my_func)
task_2 = BashOperator(
task_id="task_2",
bash_command="{{ ti.xcom_pull(task_ids='task_1') }}",
)
After:
task_1 = PythonOperator(task_id="task_1", python_callable=my_func)
task_2 = BashOperator(
task_id="task_2",
bash_command=task_1.output,
)
The same pattern applies to TaskFlow-decorated functions and covers both
`ti.xcom_pull(...)` and `task_instance.xcom_pull(...)` forms.
Rationale:
- Explicit dependencies: Using `.output` makes the data dependency visible to
the DAG parser, enabling correct scheduling and rendering in the Graph view —
whereas a Jinja template string is opaque to the parser until runtime.
- Better IDE support: `.output` provides autocompletion and go-to-definition;
template strings do not.
- Refactoring safety: Renaming a task with `.output` references is caught by
standard tooling. A `task_ids='...'` string silently breaks.
- Consistency with TaskFlow: `.output` is already the idiomatic pattern in the
TaskFlow API, so adopting it broadly reduces two mental models down to one.
- Real-world impact: An initial ecosystem scan of the Airflow repository itself
found multiple existing violations of this pattern across provider example DAGs
(Amazon, Google, Databricks), confirming this is a common and consequential
pattern in practice.
Cases NOT affected by this proposal:
- Mixed-content strings such as `"echo {{ ti.xcom_pull(task_ids='task_1') }}"`
— the template is part of a larger string, so `.output` cannot be a drop-in
replacement.
- Non-default `key` arguments such as `xcom_pull(task_ids='task_1',
key='my_key')` — these target a specific named XCom push, not the default
single output.
- List `task_ids` such as `xcom_pull(task_ids=['a', 'b'])` — aggregating
multiple outputs is outside the scope of `.output`.
Call for Consensus:
Please let me know if you have concerns, questions, or support.
Thank you,
Dev-iL
See Also:
1. https://github.com/apache/airflow/issues/43176#issuecomment-2667826944 —
original proposal in the static checks tracking issue
2. https://github.com/astral-sh/ruff/pull/23583 — Draft Ruff PR implementing
AIR004 (`airflow-xcom-pull-in-template-string`)
3. https://github.com/apache/airflow/pull/62529 — Merged PR updating some of
the matching patterns in the codebase
> On Mar 17, 2026, at 6:11 PM, Dev-iL <[email protected]> wrote:
>
> |Hello everyone,|
> ||
> |I'd like to initiate a vote on another new DAG authorship best practice.|
> ||
> |## Proposal|
> ||
> |When passing the output of one task to another via a Jinja template string
> containing a single `xcom_pull` call — such as `"{{
> ti.xcom_pull(task_ids='some_task') }}"` — the **`.output` attribute on the
> task object** should be preferred instead.|
> ||
> |### Before|
> ||
> |```python|
> |task_1 = PythonOperator(task_id="task_1", python_callable=my_func)|
> |task_2 = BashOperator(|
> |task_id="task_2",|
> |bash_command="{{ ti.xcom_pull(task_ids='task_1') }}",|
> |)|
> |```|
> ||
> |### After|
> ||
> |```python|
> |task_1 = PythonOperator(task_id="task_1", python_callable=my_func)|
> |task_2 = BashOperator(|
> |task_id="task_2",|
> |bash_command=task_1.output,|
> |)|
> |```|
> ||
> |The same pattern applies to TaskFlow-decorated functions and covers both
> `ti.xcom_pull(...)` and `task_instance.xcom_pull(...)` forms.|
> ||
> |## Rationale|
> ||
> |- **Explicit dependencies:** Using `.output` makes the data dependency
> visible to the DAG parser, enabling correct scheduling and rendering in the
> Graph view — whereas a Jinja template string is opaque to the parser until
> runtime.|
> |- **Better IDE support:** `.output` provides autocompletion and
> go-to-definition; template strings do not.|
> |- **Refactoring safety:** Renaming a task with `.output` references is
> caught by standard tooling. A `task_ids='...'` string silently breaks.|
> |- **Consistency with TaskFlow:** `.output` is already the idiomatic pattern
> in the TaskFlow API, so adopting it broadly reduces two mental models down to
> one.|
> |- **Real-world impact:** An initial ecosystem scan of the Airflow repository
> itself found multiple existing violations of this pattern across provider
> example DAGs (Amazon, Google, Databricks), confirming this is a common and
> consequential pattern in practice.|
> ||
> |## Cases NOT affected by this proposal|
> ||
> |This best practice deliberately scopes to the unambiguous case of a full,
> standalone `xcom_pull` template:|
> ||
> |- **Mixed-content strings** such as `"echo {{
> ti.xcom_pull(task_ids='task_1') }}"` — the template is part of a larger
> string, so `.output` cannot be a drop-in replacement.|
> |- **Non-default `key` arguments** such as `xcom_pull(task_ids='task_1',
> key='my_key')` — these target a specific named XCom push, not the default
> single output.|
> |- **List `task_ids`** such as `xcom_pull(task_ids=['a', 'b'])` — aggregating
> multiple outputs is outside the scope of `.output`.|
> ||
> |## Call for Consensus|
> ||
> |Please let me know if you have concerns, questions, or support.|
> ||
> |Thank you,|
> |Dev-iL|
> ||
> |---|
> ||
> |## See Also|
> ||
> |1. [apache/airflow#43176
> (comment)](https://github.com/apache/airflow/issues/43176#issuecomment-2667826944)
> — original proposal in the static checks tracking issue|
> |2. [astral-sh/ruff#23583](https://github.com/astral-sh/ruff/pull/23583) —
> Draft Ruff PR implementing `AIR004` (`airflow-xcom-pull-in-template-string`)|
> |3.
> [||apache/airflow#||62529||](https://github.com/apache/airflow/pull/62529)||—
> Merged PR updating some of the matching patterns in the codebase.|
>
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]