forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Tao Feng
authored and
Chris Fei
committed
Jan 23, 2019
1 parent
adcacfc
commit bba5721
Showing
3 changed files
with
154 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# 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. | ||
|
||
from airflow.sensors.base_sensor_operator import BaseSensorOperator | ||
from airflow.utils.decorators import apply_defaults | ||
|
||
|
||
class PythonSensor(BaseSensorOperator): | ||
""" | ||
Waits for a Python callable to return True. | ||
User could put input argument in templates_dict | ||
e.g templates_dict = {'start_ds': 1970} | ||
and access the argument by calling `kwargs['templates_dict']['start_ds']` | ||
in the the callable | ||
: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() | ||
|
||
@apply_defaults | ||
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 | ||
|
||
self.log.info("Poking callable: " + str(self.python_callable)) | ||
return_value = self.python_callable(*self.op_args, **self.op_kwargs) | ||
return bool(return_value) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
# -*- coding: utf-8 -*- | ||
# | ||
# 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. | ||
|
||
|
||
import unittest | ||
|
||
from airflow import DAG, configuration | ||
from airflow.contrib.sensors.python_sensor import PythonSensor | ||
from airflow.exceptions import AirflowSensorTimeout | ||
from airflow.models import DagBag | ||
from airflow.utils.timezone import datetime | ||
|
||
|
||
DEFAULT_DATE = datetime(2015, 1, 1) | ||
TEST_DAG_ID = 'python_sensor_dag' | ||
DEV_NULL = '/dev/null' | ||
|
||
|
||
class PythonSensorTests(unittest.TestCase): | ||
|
||
def setUp(self): | ||
configuration.load_test_config() | ||
self.dagbag = DagBag( | ||
dag_folder=DEV_NULL, | ||
include_examples=True | ||
) | ||
self.args = { | ||
'owner': 'airflow', | ||
'start_date': DEFAULT_DATE | ||
} | ||
dag = DAG(TEST_DAG_ID, default_args=self.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) |