Would using Variable be an alternative?

From your description, it feels like your team is kind of using XCom as a 
key-value store, which Variable is designed to be. It’s also available on 
Airflow 2 so the migration should be straightforward.

 TP

--
Sent from my iPhone

> On 17 Apr 2026, at 00:37, Foldvari, Gyorgy via dev <[email protected]> 
> wrote:
> 
> Hi Ash,
> 
> To be very specific: My team is heavily relying on the Airflow 2 xcom_pull 
> behavior when the key parameter is defined but the task_ids parameter is not 
> defined. In this case Airflow 2 returns the most recent xcom value pushed by 
> any task in that DAG run using that key, or None if no value has been pushed 
> with that key.
> The change of the behavior introduced in Airflow 3 without any equivalent 
> alternative is a blocker for us to migrate to Airflow 3.
> 
> One option could be to restore the original behavior. I see that there will 
> be no consensus on this, and I understand the reasons. Any other solution 
> would help if it can be used in template fields as well without the need to 
> change the code of classic Operators themselves.
> 
> As far as I see we already have good proposals.
> 
> Thanks,
> Gyorgy Foldvari
> 
> 
> Information Classification: GENERAL
> -----Original Message-----
> From: Ash Berlin-Taylor <[email protected]>
> Sent: Thursday, April 16, 2026 15:48
> To: [email protected]
> Subject: Re: [DISCUSS] Reconsidering `xcom_pull(task_ids=None)` behavior 
> change in Airflow 3
> 
> [You don't often get email from [email protected]. Learn why this is important 
> at https://aka.ms/LearnAboutSenderIdentification ]
> 
> CAUTION: This email originated from outside of the organization. Do not click 
> links or open attachments unless you recognize the sender and know the 
> content is safe.
> 
> 
> Gyorgy,
> 
> Your original post did not mention specifics, and is talking about behaviour. 
> The thing that is most relevant is what kind of workload exists that means 
> you want this behaviour. XCom is already complicated enough once you get 
> beyond “pull this specific task” without adding even more bells and 
> whistles.
> 
> So please be as specific and real as you can in saying why you want this 
> feature.
> 
> Thanks,
> Ash
> 
>> On 15 Apr 2026, at 21:25, Jens Scheffler <[email protected]> wrote:
>> 
>> I do not have the use case and prefer explicit pinning to the task where I 
>> want XCom from. But maybe my/our use cases are structured and repeatable 
>> that the questions does not hold.
>> 
>> So if there is the demand for a non-deterministic retrieval of the "OLDEST" 
>> / "NEWEST" (single) entry I think using the constant classes is a good 
>> approach not to overload the function signature. If this is well documented 
>> in PyDoc then it is accessible. Would be OK for me.
>> 
>> Otherwise a more generic solution could be providing an API which accepts a 
>> list of "candidates" to poll meta data and get a lis and this list is 
>> including dates. Then some logic could sort by date and retrieve whatever es 
>> desired... if there are more niece use cases this might be an alternative. 
>> But the proposal is lightweight.
>> 
>>> On 15.04.26 17:03, Jarek Potiuk wrote:
>>> I'd wait for others to comment. I want to see what others think.
>>> 
>>> On Wed, Apr 15, 2026 at 4:51 PM Foldvari, Gyorgy via dev <
>>> [email protected]> wrote:
>>> 
>>>> If xcom_pull(task_ids=XCom.ANY, key=”sample”) would return with a
>>>> list, that would not be the same result what was produced by Airflow
>>>> 2. Airflow 2 returns with the most recent value and not with a list.
>>>> I am wondering what would make more sense:
>>>> 
>>>>  1.  Return with one value, the most recent one (on None) instead
>>>> of a list, copying Airflow 2 behavior.
>>>>  2.  Return with a list but also defining a new optional parameter
>>>> to override this behavior (XCom.FIRST_OLDEST, XCom.FIRST_NEWEST).
>>>> I am struggling with coming up a valid use case where getting back a
>>>> list of values without any further information would make sense. So
>>>> probably we should go with the simpler first option.
>>>> 
>>>> 
>>>> Information Classification: GENERAL
>>>> 
>>>> On 2026/04/15 14:35:35 Jarek Potiuk wrote:
>>>>>> What do you think about this solution?
>>>>>> ti.xcom_pull accepting ‘*’ or rather a sentinel value as task_ids
>>>>>> to
>>>>> explicitly say that ANY task_id should be considered. The result
>>>>> would be all the values pushed to xcom by any task in that dag run
>>>>> as a list,
>>>> sorted
>>>>> by XCOM timestamp.
>>>>>> ti.xcom_pull(task_ids= ANY, key=”example”)
>>>>> 
>>>>> I'd rather (if others see a value for it) have some explicit
>>>>> sentinel (XCom.FiRST_OLDEST, XCom.FIRST_NEWEST - or whatever makes
>>>>> sense for
>>>> those).
>>>>>> The only challenge is how to use this in classic Operators’
>>>>>> templated
>>>>> fields.
>>>>> 
>>>>> Those sentinels could be easily exposed to JINJA and predefined there.
>>>>> 
>>>>> It might of course be over-the-top, and some might say "ugly" - but
>>>>> if it makes it easier for people to migrate to 3 and would not make
>>>>> it more difficult for maintenance, I am all for it.
>>>>> 
>>>>> J.
>>>>> 
>>>>> 
>>>>> On Wed, Apr 15, 2026 at 4:22 PM Foldvari, Gyorgy via dev <
>>>>> [email protected]> wrote:
>>>>> 
>>>>>> What do you think about this solution?
>>>>>> ti.xcom_pull accepting ‘*’ or rather a sentinel value as task_ids
>>>>>> to explicitly say that ANY task_id should be considered. The
>>>>>> result would
>>>> be
>>>>>> all the values pushed to xcom by any task in that dag run as a
>>>>>> list,
>>>> sorted
>>>>>> by XCOM timestamp.
>>>>>> ti.xcom_pull(task_ids= ANY, key=”example”) The only challenge is
>>>>>> how to use this in classic Operators’ templated fields.
>>>>>> 
>>>>>> 
>>>>>> Information Classification: GENERAL
>>>>>> 
>>>>>> On 2026/04/15 13:34:58 Jarek Potiuk wrote:
>>>>>>>> This was ambiguous regarding the value you wanted to retrieve
>>>>>>>> and
>>>> the
>>>>>>> intended change.
>>>>>>> 
>>>>>>> While I see why we cannot (should not) bring back the "exact"
>>>> behaviour,
>>>>>> I
>>>>>>> think we could - potentially at least - make it non-ambiguous and
>>>>>>> predictable by defining the semantics and ordering and selecting
>>>>>>> the
>>>>>> first
>>>>>>> one. And - I think - we do not **have** to bring it back in this
>>>>>> particular
>>>>>>> method. Maybe we can just a new method call with defined ordering
>>>>>>> and semantics - behaving predictably, similar to Airflow 2—and
>>>>>>> clearly
>>>>>> defined
>>>>>>> semantically. That would at least give people an easier way to
>>>> migrate?
>>>>>>> While I think the cat if out-of-the bag and we cannot truly
>>>>>>> revert
>>>> the
>>>>>>> change (because that would again potentially affect 3.0 - 3.2
>>>> users)  -
>>>>>> but
>>>>>>> we could at least make it easier for people to cope with it
>>>>>>> without
>>>> too
>>>>>>> much hassle while waiting for the task state to be available in
>>>>>>> this particular case?
>>>>>>> 
>>>>>>> Just a thought I had - ... listen to your users and do things
>>>>>>> easier
>>>> for
>>>>>>> them - without breaking our SemVer promises.
>>>>>>> 
>>>>>>> J.
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> On Wed, Apr 15, 2026 at 3:20 PM Foldvari, Gyorgy via dev <
>>>>>>> [email protected]> wrote:
>>>>>>> 
>>>>>>>> I do not want to use XCom for managing task state. It was just a
>>>> very
>>>>>>>> simple - and seemingly misleading - example to explain the
>>>>>>>> original behavior.
>>>>>>>> 
>>>>>>>> I gave details about valid use cases and issues caused by this
>>>> change
>>>>>> in
>>>>>>>> the behavior, in my original post. Those are not addressed by
>>>>>>>> the
>>>> AIP
>>>>>> you
>>>>>>>> are referring to.
>>>>>>>> 
>>>>>>>> But that AIP would definitely address another valid use case
>>>>>>>> what
>>>> I am
>>>>>>>> missing especially for implementing stateful sensors, so I
>>>>>>>> really
>>>> hope
>>>>>> it
>>>>>>>> goes through and gets implemented.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Information Classification: GENERAL -----Original Message-----
>>>>>>>> From: Ash Berlin-Taylor <[email protected]>
>>>>>>>> Sent: Wednesday, April 15, 2026 14:49
>>>>>>>> To: [email protected]
>>>>>>>> Subject: Re: [DISCUSS] Reconsidering `xcom_pull(task_ids=None)`
>>>>>> behavior
>>>>>>>> change in Airflow 3
>>>>>>>> 
>>>>>>>> [You don't often get email from [email protected]. Learn why this
>>>>>>>> is important at https://aka.ms/LearnAboutSenderIdentification ]
>>>>>>>> 
>>>>>>>> CAUTION: This email originated from outside of the organization.
>>>> Do not
>>>>>>>> click links or open attachments unless you recognize the sender
>>>>>>>> and
>>>>>> know
>>>>>>>> the content is safe.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> This was ambiguous as to what value you wanted to get, and an
>>>> intended
>>>>>>>> change.
>>>>>>>> 
>>>>>>>> If you want this sort of behaviour, then you probably want to
>>>>>>>> look
>>>> at
>>>> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcw
>>>> iki.apache.org%2Fconfluence%2Fdisplay%2FAIRFLOW%2FAIP-103%253A%2BTas
>>>> k%2BState%2BManagement&data=05%7C02%7CGyorgy.Foldvari%40msci.com%7C3
>>>> f431909517b460c5ea708de9bbed80d%7C7a9376d47c43480f82baa090647f651d%7
>>>> C0%7C0%7C639119441164612772%7CUnknown%7CTWFpbGZsb3d8eyJFbXB0eU1hcGki
>>>> OnRydWUsIlYiOiIwLjAuMDAwMCIsIlAiOiJXaW4zMiIsIkFOIjoiTWFpbCIsIldUIjoy
>>>> fQ%3D%3D%7C0%7C%7C%7C&sdata=yNOVBbNTbuEw6Niw%2B4ilPG33YqSAp8bEb%2Bf2
>>>> UDq6xSI%3D&reserved=0
>>>>>>>> which provides a dedicated way to manage state without many of
>>>>>>>> the
>>>>>> quirks
>>>>>>>> of XCom interface as it stands today. Reading between the lines,
>>>>>>>> I
>>>>>> think
>>>>>>>> this API describes what you want?
>>>>>>>> 
>>>>>>>> -ash
>>>>>>>> 
>>>>>>>>> On 15 Apr 2026, at 12:45, Foldvari, Gyorgy via dev <
>>>>>>>> [email protected]> wrote:
>>>>>>>>> I see where the confusion is coming from, it is my mistake.
>>>>>>>>> Sorry
>>>>>> about
>>>>>>>> that.
>>>>>>>>> To clarify, I am taking about the use case where the key
>>>> parameter is
>>>>>>>> passed but the task_ids parameter is not or it is None.
>>>>>>>>> -----Original Message-----
>>>>>>>>> From: Foldvari, Gyorgy via dev <[email protected]>
>>>>>>>>> Sent: Wednesday, April 15, 2026 13:42
>>>>>>>>> To: [email protected]
>>>>>>>>> Cc: Foldvari, Gyorgy <[email protected]>
>>>>>>>>> Subject: RE: Re: [DISCUSS] Reconsidering
>>>> `xcom_pull(task_ids=None)`
>>>>>>>>> behavior change in Airflow 3
>>>>>>>>> 
>>>>>>>>> CAUTION: This email originated from outside of the organization.
>>>> Do
>>>>>> not
>>>>>>>> click links or open attachments unless you recognize the sender
>>>>>>>> and
>>>>>> know
>>>>>>>> the content is safe.
>>>>>>>>> 
>>>>>>>>> The original behavior is to return the most recent value put by
>>>> any
>>>>>>>> upstream task of the same run. Not all the values, only the
>>>>>>>> recent
>>>> one.
>>>>>>>>> Supposing that there are multiple tasks pushing values to XCom
>>>>>>>>> in
>>>>>> this
>>>>>>>> order:
>>>>>>>>> Task1: ti.xcom_push(key="example", value=1)
>>>>>>>>> Task2: ti.xcom_push(key="example", value=2)
>>>>>>>>> Task3: ti.xcom_push(key="example", value=3) Then in a
>>>>>>>>> downstream
>>>> task
>>>>>>>> ti.comm_pull(key="example") returns 3.
>>>>>>>>> I do not propose to change this behavior.
>>>>>>>>> 
>>>>>>>>> On 2026/04/14 16:09:05 Daniel Standish via dev wrote:
>>>>>>>>>> So the behavior before would be that it would return all xcom
>>>> values
>>>>>>>>>> that were emitted from the present run?
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> ________________________________
>>>>>>>>> 
>>>>>>>>> Information regarding MSCI's processing of personal data may be
>>>> found
>>>>>>>>> at http://www.msci.com/privacy-pledge. This email message and
>>>> any
>>>>>>>> attachments
>>>>>>>>> are for the sole use of the intended recipients and may contain
>>>>>>>>> proprietary and/or confidential information which may be
>>>> privileged
>>>>>> or
>>>>>>>>> otherwise protected from disclosure. Any unauthorized review,
>>>> use,
>>>>>>>>> disclosure or distribution is prohibited. All rights and
>>>> remedies are
>>>>>>>>> reserved. If you are not an intended recipient, please contact
>>>> the
>>>>>>>>> sender by reply email and destroy the original message and any
>>>> copies
>>>>>>>>> of the message as well as any attachments to the original
>>>> message.
>>>>>>>>> Local registered entity information:
>>>>>>>>> https://www.msci.com/local-registered-entities
>>>>>>>>> 
>>>>>>>>> 
>>>> --------------------------------------------------------------------
>>>> -
>>>>>>>>> To unsubscribe, e-mail: [email protected]
>>>>>>>>> For additional commands, e-mail: [email protected]
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> ________________________________
>>>>>>>> 
>>>>>>>> Information regarding MSCI’s processing of personal data may be
>>>> found
>>>>>> at
>>>>>>>> http://www.msci.com/privacy-pledge. This email message and any
>>>>>>>> attachments are for the sole use of the intended recipients and
>>>>>>>> may
>>>>>> contain
>>>>>>>> proprietary and/or confidential information which may be
>>>> privileged or
>>>>>>>> otherwise protected from disclosure. Any unauthorized review,
>>>>>>>> use, disclosure or distribution is prohibited. All rights and
>>>>>>>> remedies
>>>> are
>>>>>>>> reserved. If you are not an intended recipient, please contact
>>>>>>>> the
>>>>>> sender
>>>>>>>> by reply email and destroy the original message and any copies
>>>>>>>> of
>>>> the
>>>>>>>> message as well as any attachments to the original message.
>>>>>>>> Local registered entity information:
>>>>>>>> https://www.msci.com/local-registered-entities
>>>>>>>> 
>>>>>> ________________________________
>>>>>> 
>>>>>> Information regarding MSCI’s processing of personal data may be
>>>>>> found
>>>> at
>>>>>> www.msci.com/privacy-pledge. This email message and any
>>>>>> attachments
>>>> are
>>>>>> for the sole use of the intended recipients and may contain
>>>>>> proprietary and/or confidential information which may be
>>>>>> privileged or otherwise protected from disclosure. Any
>>>>>> unauthorized review, use, disclosure or distribution is
>>>>>> prohibited. All rights and remedies are reserved. If
>>>> you
>>>>>> are not an intended recipient, please contact the sender by reply
>>>> email and
>>>>>> destroy the original message and any copies of the message as well
>>>>>> as
>>>> any
>>>>>> attachments to the original message. Local registered entity
>>>> information:
>>>>>> https://www.msci.com/local-registered-entities
>>>>>> 
>>>> ________________________________
>>>> 
>>>> Information regarding MSCI’s processing of personal data may be
>>>> found at www.msci.com/privacy-pledge. This email message and any
>>>> attachments are for the sole use of the intended recipients and may
>>>> contain proprietary and/or confidential information which may be
>>>> privileged or otherwise protected from disclosure. Any unauthorized
>>>> review, use, disclosure or distribution is prohibited. All rights
>>>> and remedies are reserved. If you are not an intended recipient,
>>>> please contact the sender by reply email and destroy the original
>>>> message and any copies of the message as well as any attachments to the 
>>>> original message. Local registered entity information:
>>>> https://www.msci.com/local-registered-entities
>>>> 
>> 
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: [email protected]
>> For additional commands, e-mail: [email protected]
>> 
> 
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: [email protected]
> For additional commands, e-mail: [email protected]
> 
> 
> ________________________________
> 
> Information regarding MSCI’s processing of personal data may be found at 
> www.msci.com/privacy-pledge. This email message and any attachments are for 
> the sole use of the intended recipients and may contain proprietary and/or 
> confidential information which may be privileged or otherwise protected from 
> disclosure. Any unauthorized review, use, disclosure or distribution is 
> prohibited. All rights and remedies are reserved. If you are not an intended 
> recipient, please contact the sender by reply email and destroy the original 
> message and any copies of the message as well as any attachments to the 
> original message. Local registered entity information: 
> https://www.msci.com/local-registered-entities
> ТÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÒÐÐ¥FòVç7V'67&–&RÂRÖÖ–âFWb×Vç7V'67&–&T—&fÆ÷ræ6†Ræ÷&pФf÷"FF—F–öæÂ6öÖÖæG2ÂRÖÖ–âFWbÖ†VÇ—&fÆ÷ræ6†Ræ÷&pÐ

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to