Repository: incubator-airflow
Updated Branches:
  refs/heads/master df8e331ef -> 92f44f5b5


[AIRFLOW-196] Fix bug that exception is not handled in HttpSensor

Dear Airflow Maintainers,

Please accept this PR that addresses the following issues:
- [*AIRFLOW-196*](https://issues.apache.org/jira/browse/AIRFLOW-196)

If exception happens in poke function in HttpSensor, it is not
well handled that make the sensor finish successfully, which is
incorrect obviously.

Author: Junwei Wang <i.junwei.w...@gmail.com>

Closes #1561 from junwei-wang/master.


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/92f44f5b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/92f44f5b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/92f44f5b

Branch: refs/heads/master
Commit: 92f44f5b5b47fdedf7e5908f24ee28bf242b4634
Parents: df8e331
Author: Junwei Wang <i.junwei.w...@gmail.com>
Authored: Tue May 31 20:47:04 2016 -0700
Committer: Chris Riccomini <chr...@wepay.com>
Committed: Tue May 31 20:47:04 2016 -0700

----------------------------------------------------------------------
 airflow/operators/sensors.py |  2 ++
 tests/operators/sensor.py    | 39 +++++++++++++++++++++++++++++++++++++++
 2 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/92f44f5b/airflow/operators/sensors.py
----------------------------------------------------------------------
diff --git a/airflow/operators/sensors.py b/airflow/operators/sensors.py
index 31ef7b3..569b4d7 100644
--- a/airflow/operators/sensors.py
+++ b/airflow/operators/sensors.py
@@ -516,4 +516,6 @@ class HttpSensor(BaseSensorOperator):
             if ae.message.startswith("404"):
                 return False
 
+            raise ae
+
         return True

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/92f44f5b/tests/operators/sensor.py
----------------------------------------------------------------------
diff --git a/tests/operators/sensor.py b/tests/operators/sensor.py
new file mode 100644
index 0000000..45d4b81
--- /dev/null
+++ b/tests/operators/sensor.py
@@ -0,0 +1,39 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed 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 os
+import unittest
+
+from airflow.operators import HttpSensor
+from airflow.exceptions import AirflowException
+
+
+class HttpSensorTests(unittest.TestCase):
+
+    def test_poke_exception(self):
+        """
+        Exception occurs in poke function should not be ignored.
+        """
+        def resp_check(resp):
+            raise AirflowException('AirflowException raised here!')
+
+        task = HttpSensor(
+            task_id='http_sensor_poke_exception',
+            http_conn_id='http_default',
+            endpoint='',
+            params={},
+            response_check=resp_check,
+            poke_interval=5)
+        with self.assertRaisesRegexp(AirflowException, 'AirflowException 
raised here!'):
+            task.execute(None)

Reply via email to