You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2019/01/11 09:23:04 UTC

[GitHub] Fokko closed pull request #4056: [AIRFLOW-3207] option to stop task pushing result to xcom

Fokko closed pull request #4056: [AIRFLOW-3207] option to stop task pushing result to xcom
URL: https://github.com/apache/airflow/pull/4056
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py
index aa93f5cb88..611ed99bc7 100755
--- a/airflow/models/__init__.py
+++ b/airflow/models/__init__.py
@@ -1431,7 +1431,7 @@ def signal_handler(signum, frame):
                     result = task_copy.execute(context=context)
 
                 # If the task returns a result, push an XCom containing it
-                if result is not None:
+                if task_copy.do_xcom_push and result is not None:
                     self.xcom_push(key=XCOM_RETURN_KEY, value=result)
 
                 task_copy.post_execute(context=context, result=result)
@@ -2231,6 +2231,9 @@ class derived from this one results in the creation of a task object,
             )
 
     :type executor_config: dict
+    :param do_xcom_push: if True, an XCom is pushed containing the Operator's
+        result
+    :type do_xcom_push: bool
     """
 
     # For derived classes to define which fields will get jinjaified
@@ -2285,6 +2288,7 @@ def __init__(
             run_as_user=None,
             task_concurrency=None,
             executor_config=None,
+            do_xcom_push=True,
             inlets=None,
             outlets=None,
             *args,
@@ -2369,6 +2373,7 @@ def __init__(
         self.run_as_user = run_as_user
         self.task_concurrency = task_concurrency
         self.executor_config = executor_config or {}
+        self.do_xcom_push = do_xcom_push
 
         # Private attributes
         self._upstream_task_ids = set()
@@ -2422,6 +2427,7 @@ def __init__(
             'on_failure_callback',
             'on_success_callback',
             'on_retry_callback',
+            'do_xcom_push',
         }
 
     def __eq__(self, other):
diff --git a/tests/models.py b/tests/models.py
index 2e8a139eb0..8ebf8720ce 100644
--- a/tests/models.py
+++ b/tests/models.py
@@ -2429,6 +2429,32 @@ def test_xcom_pull_different_execution_date(self):
                                       include_prior_dates=True),
                          value)
 
+    def test_xcom_push_flag(self):
+        """
+        Tests the option for Operators to push XComs
+        """
+        value = 'hello'
+        task_id = 'test_no_xcom_push'
+        dag = models.DAG(dag_id='test_xcom')
+
+        # nothing saved to XCom
+        task = PythonOperator(
+            task_id=task_id,
+            dag=dag,
+            python_callable=lambda: value,
+            do_xcom_push=False,
+            owner='airflow',
+            start_date=datetime.datetime(2017, 1, 1)
+        )
+        ti = TI(task=task, execution_date=datetime.datetime(2017, 1, 1))
+        ti.run()
+        self.assertEqual(
+            ti.xcom_pull(
+                task_ids=task_id, key=models.XCOM_RETURN_KEY
+            ),
+            None
+        )
+
     def test_post_execute_hook(self):
         """
         Test that post_execute hook is called with the Operator's result.


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services