[
https://issues.apache.org/jira/browse/AIRFLOW-850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16726474#comment-16726474
]
ASF GitHub Bot commented on AIRFLOW-850:
----------------------------------------
feng-tao closed pull request #2058: [AIRFLOW-850] Add a PythonSensor
URL: https://github.com/apache/incubator-airflow/pull/2058
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/operators/sensors.py b/airflow/operators/sensors.py
index 44a97e00c1..bf02335a95 100644
--- a/airflow/operators/sensors.py
+++ b/airflow/operators/sensors.py
@@ -679,3 +679,57 @@ def poke(self, context):
raise ae
return True
+
+class PythonSensor(BaseSensorOperator):
+ """
+ Waits for a Python callable to return True
+
+ :param python_callable: A reference to an object that is callable
+ :type python_callable: python callable
+ :param op_kwargs: a dictionary of keyword arguments that will get unpacked
+ in your function
+ :type op_kwargs: dict
+ :param op_args: a list of positional arguments that will get unpacked when
+ calling your callable
+ :type op_args: list
+ :param provide_context: if set to true, Airflow will pass a set of
+ keyword arguments that can be used in your function. This set of
+ kwargs correspond exactly to what you can use in your jinja
+ templates. For this to work, you need to define `**kwargs` in your
+ function header.
+ :type provide_context: bool
+ :param templates_dict: a dictionary where the values are templates that
+ will get templated by the Airflow engine sometime between
+ ``__init__`` and ``execute`` takes place and are made available
+ in your callable's context after the template has been applied
+ :type templates_dict: dict of str
+ """
+
+ template_fields = ('templates_dict',)
+ template_ext = tuple()
+
+ def __init__(
+ self,
+ python_callable,
+ op_args=None,
+ op_kwargs=None,
+ provide_context=False,
+ templates_dict=None,
+ *args, **kwargs):
+ super(PythonSensor, self).__init__(*args, **kwargs)
+ self.python_callable = python_callable
+ self.op_args = op_args or []
+ self.op_kwargs = op_kwargs or {}
+ self.provide_context = provide_context
+ self.templates_dict = templates_dict
+
+
+ def poke(self, context):
+ if self.provide_context:
+ context.update(self.op_kwargs)
+ context['templates_dict'] = self.templates_dict
+ self.op_kwargs = context
+
+ logging.info("Poking callable: " + str(self.python_callable))
+ return_value = self.python_callable(*self.op_args, **self.op_kwargs)
+ return bool(return_value)
diff --git a/tests/operators/sensors.py b/tests/operators/sensors.py
index e77216b580..2633e4c41b 100644
--- a/tests/operators/sensors.py
+++ b/tests/operators/sensors.py
@@ -22,7 +22,7 @@
from datetime import datetime, timedelta
from airflow import DAG, configuration
-from airflow.operators.sensors import HttpSensor, BaseSensorOperator,
HdfsSensor
+from airflow.operators.sensors import HttpSensor, BaseSensorOperator,
HdfsSensor, PythonSensor
from airflow.utils.decorators import apply_defaults
from airflow.exceptions import (AirflowException,
AirflowSensorTimeout,
@@ -181,3 +181,38 @@ def test_legacy_file_does_not_exists(self):
# Then
with self.assertRaises(AirflowSensorTimeout):
task.execute(None)
+
+class PythonSensorTests(unittest.TestCase):
+
+ def setUp(self):
+ configuration.load_test_config()
+ args = {
+ 'owner': 'airflow',
+ 'start_date': DEFAULT_DATE
+ }
+ dag = DAG(TEST_DAG_ID, default_args=args)
+ self.dag = dag
+
+ def test_python_sensor_true(self):
+ t = PythonSensor(
+ task_id='python_sensor_check_true',
+ python_callable=lambda: True,
+ dag=self.dag)
+ t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE,
ignore_ti_state=True)
+
+ def test_python_sensor_false(self):
+ t = PythonSensor(
+ task_id='python_sensor_check_false',
+ timeout=1,
+ python_callable=lambda: False,
+ dag=self.dag)
+ with self.assertRaises(AirflowSensorTimeout):
+ t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE,
ignore_ti_state=True)
+
+ def test_python_sensor_raise(self):
+ t = PythonSensor(
+ task_id='python_sensor_check_raise',
+ python_callable=lambda: 1/0,
+ dag=self.dag)
+ with self.assertRaises(ZeroDivisionError):
+ t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE,
ignore_ti_state=True)
----------------------------------------------------------------
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:
[email protected]
> Airflow should support a general purpose PythonSensor
> -----------------------------------------------------
>
> Key: AIRFLOW-850
> URL: https://issues.apache.org/jira/browse/AIRFLOW-850
> Project: Apache Airflow
> Issue Type: Improvement
> Components: operators
> Affects Versions: 1.8.0
> Reporter: Daniel Gies
> Assignee: Daniel Gies
> Priority: Major
>
> Today I found myself trying to use a sensor to postpone execution until data
> for the current execution date appeared in a file. It occurred to me that
> having a general purpose PythonSensor would allow developers to use the
> sensor paradigm with arbitrary code.
> We should add a PythonSensor to the core sensors module which takes a
> python_callable and optional args like the PythonOperator does.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)