You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by di...@apache.org on 2020/10/16 22:19:39 UTC
[airflow] branch master updated: Raises a warning for
provide_context instead of killing the task (#11597)
This is an automated email from the ASF dual-hosted git repository.
dimberman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git
The following commit(s) were added to refs/heads/master by this push:
new 00dd758 Raises a warning for provide_context instead of killing the task (#11597)
00dd758 is described below
commit 00dd7586fb6b41bff2b89b9008742c7b34cd4fdd
Author: Daniel Imberman <da...@gmail.com>
AuthorDate: Fri Oct 16 15:18:55 2020 -0700
Raises a warning for provide_context instead of killing the task (#11597)
* raises a warning for provide_context instead of killing the task
* Update airflow/operators/python.py
Co-authored-by: Kaxil Naik <ka...@gmail.com>
* static checks
Co-authored-by: Kaxil Naik <ka...@gmail.com>
---
airflow/operators/python.py | 5 +++++
tests/operators/test_python.py | 25 +++++++++++++++++++++++++
2 files changed, 30 insertions(+)
diff --git a/airflow/operators/python.py b/airflow/operators/python.py
index 58a1095..1be50e2 100644
--- a/airflow/operators/python.py
+++ b/airflow/operators/python.py
@@ -22,6 +22,7 @@ import pickle
import re
import sys
import types
+import warnings
from inspect import signature
from itertools import islice
from tempfile import TemporaryDirectory
@@ -85,6 +86,10 @@ class PythonOperator(BaseOperator):
templates_exts: Optional[List[str]] = None,
**kwargs
) -> None:
+ if kwargs.get("provide_context"):
+ warnings.warn("provide_context is deprecated as of 2.0 and is no longer required",
+ DeprecationWarning, stacklevel=2)
+ kwargs.pop('provide_context', None)
super().__init__(**kwargs)
if not callable(python_callable):
raise AirflowException('`python_callable` param must be callable')
diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py
index 91529ffb..a61b9a1 100644
--- a/tests/operators/test_python.py
+++ b/tests/operators/test_python.py
@@ -275,6 +275,31 @@ class TestPythonOperator(TestPythonBase):
python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
self.assertTrue('dag' in context.exception, "'dag' not found in the exception")
+ def test_provide_context_does_not_fail(self):
+ """
+ ensures that provide_context doesn't break dags in 2.0
+ """
+ self.dag.create_dagrun(
+ run_type=DagRunType.MANUAL,
+ execution_date=DEFAULT_DATE,
+ start_date=DEFAULT_DATE,
+ state=State.RUNNING,
+ external_trigger=False,
+ )
+
+ def func(custom, dag):
+ self.assertEqual(1, custom, "custom should be 1")
+ self.assertIsNotNone(dag, "dag should be set")
+
+ python_operator = PythonOperator(
+ task_id='python_operator',
+ op_kwargs={'custom': 1},
+ python_callable=func,
+ provide_context=True,
+ dag=self.dag
+ )
+ python_operator.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE)
+
def test_context_with_conflicting_op_args(self):
self.dag.create_dagrun(
run_type=DagRunType.MANUAL,