amoghrajesh opened a new pull request, #53814:
URL: https://github.com/apache/airflow/pull/53814

   <!--
    Licensed to the Apache Software Foundation (ASF) under one
    or more contributor license agreements.  See the NOTICE file
    distributed with this work for additional information
    regarding copyright ownership.  The ASF licenses this file
    to you under the Apache License, Version 2.0 (the
    "License"); you may not use this file except in compliance
    with the License.  You may obtain a copy of the License at
   
      http://www.apache.org/licenses/LICENSE-2.0
   
    Unless required by applicable law or agreed to in writing,
    software distributed under the License is distributed on an
    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    KIND, either express or implied.  See the License for the
    specific language governing permissions and limitations
    under the License.
    -->
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   closes: https://github.com/apache/airflow/issues/53432
   
   
   ## Problem
   
   The `BaseXCom.get_all()` method was directly calling deserialize() from the 
serialization module instead of using the overridable cls.deserialize_value() 
method. This broke custom XCom backends that rely on overriding 
`deserialize_value()` to implement custom deserialization logic.
   
   
   Example of broken behavior:
   
   ```
   class CustomXCom(BaseXCom):
       @classmethod
       def deserialize_value(cls, result):
           # Custom logic that was being bypassed
           return f"custom_prefix:{super().deserialize_value(result)}"
   
   # get_one() worked correctly - called CustomXCom.deserialize_value()
   # get_all() was broken - bypassed CustomXCom.deserialize_value()
   ```
   
   ## Root Cause
   
   The issue comes from inconsistent deserialization patterns:
   - get_one() correctly calls `cls.deserialize_value(msg)` where msg is an 
`XComResult` with a value attribute
   - get_all() was calling `deserialize(msg.root)` directly, where `msg.root` 
is a `list[JsonValue]`
   
   The deserialize_value() method expects an object with a value attribute, but 
the sequence slice returns a flat list of serialized values.
   
   - Applied the same wrapper pattern already established in lazy_sequence.py
   - Its a lightweight wrapper that exposes `_XComValueWrapper` that provides 
`.value` as needed
   
   
   ## Testing
   
   DAG:
   ```
   from airflow import DAG
   from airflow.providers.standard.operators.python import PythonOperator
   
   def push_to_xcom(**kwargs):
       value = "Hello, XCom!"
       return value
   
   def push_to_xcom2(**kwargs):
       value = "Hello, XCom2!"
       return value
   
   def pull_from_xcom(**kwargs):
       ti = kwargs['ti']
       xcom_value = ti.xcom_pull(task_ids=["push_xcom_task", "push_xcom_task2"])
       print(f"Retrieved XCom Value: {xcom_value}")
   
   
   with DAG(
       'xcom_example',
       schedule=None,
       catchup=False,
   ) as dag:
   
       push_xcom_task = PythonOperator(
           task_id='push_xcom_task',
           python_callable=push_to_xcom,
       )
   
       push_xcom_task2 = PythonOperator(
           task_id='push_xcom_task2',
           python_callable=push_to_xcom2,
       )
   
       pull_xcom_task = PythonOperator(
           task_id='pull_xcom_task',
           python_callable=pull_from_xcom,
       )
   
       push_xcom_task >> push_xcom_task2 >> pull_xcom_task
   
   ```
   
   Write a custom xcom backend:
   ```
   from airflow.sdk.bases.xcom import BaseXCom
   
   class MyXCom(BaseXCom):
       @classmethod
       def deserialize_value(self, value):
           return {"from": "custom xcom"}
   
   ```
   
   Run breeze with:
   ```
   
   export PYTHONPATH="/files/plugins:$PYTHONPATH"
   export AIRFLOW__CORE__XCOM_BACKEND="myxcom.MyXCom"
   ```
   
   Validate if the right one got loaded:
   ```
   >>> from airflow.sdk.execution_time.xcom import XCom
   
   >>>
   >>> XCom
   <class 'myxcom.MyXCom'>
   >>> XCom.__name__
   'MyXCom'
   ```
   
   Run the dag:
   
   Push task
   <img width="1721" height="667" alt="image" 
src="https://github.com/user-attachments/assets/29ecfe09-b801-40d6-b34f-0f7dc2ebd080";
 />
   
   Pull task
   <img width="1721" height="667" alt="image" 
src="https://github.com/user-attachments/assets/d0d54b93-7ab3-4e6a-8c0a-47a77394a4fd";
 />
   
   
    
   
   <!-- Please keep an empty line above the dashes. -->
   ---
   **^ Add meaningful description above**
   Read the **[Pull Request 
Guidelines](https://github.com/apache/airflow/blob/main/contributing-docs/05_pull_requests.rst#pull-request-guidelines)**
 for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal 
([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals))
 is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party 
License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a 
newsfragment file, named `{pr_number}.significant.rst` or 
`{issue_number}.significant.rst`, in 
[airflow-core/newsfragments](https://github.com/apache/airflow/tree/main/airflow-core/newsfragments).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to