Repository: incubator-airflow
Updated Branches:
  refs/heads/master 868bfe4ca -> f1ff09d07


[AIRFLOW-1274][HTTPSENSOR] Rename parameter params to data

Renamed the parameter `params` to `request_params` in `HttpSensor` as it
was conflicting with `BaseOperator` `params` and causing failed DAGs
parsing.

Closes #2342 from aliceabe/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/f1ff09d0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/f1ff09d0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/f1ff09d0

Branch: refs/heads/master
Commit: f1ff09d07b344126a0a0698dad755f330b1f28d3
Parents: 868bfe4
Author: Alice Berard <alice.ber...@airbnb.com>
Authored: Fri Jun 2 17:43:36 2017 -0700
Committer: Dan Davydov <dan.davy...@airbnb.com>
Committed: Fri Jun 2 17:43:40 2017 -0700

----------------------------------------------------------------------
 airflow/example_dags/example_http_operator.py |  2 +-
 airflow/operators/sensors.py                  | 12 ++++++------
 tests/core.py                                 |  2 +-
 tests/operators/sensors.py                    |  6 +++---
 4 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f1ff09d0/airflow/example_dags/example_http_operator.py
----------------------------------------------------------------------
diff --git a/airflow/example_dags/example_http_operator.py 
b/airflow/example_dags/example_http_operator.py
index 983cc49..10fc3b1 100644
--- a/airflow/example_dags/example_http_operator.py
+++ b/airflow/example_dags/example_http_operator.py
@@ -81,7 +81,7 @@ sensor = HttpSensor(
     task_id='http_sensor_check',
     http_conn_id='http_default',
     endpoint='',
-    params={},
+    request_params={},
     response_check=lambda response: True if "Google" in response.content else 
False,
     poke_interval=5,
     dag=dag)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f1ff09d0/airflow/operators/sensors.py
----------------------------------------------------------------------
diff --git a/airflow/operators/sensors.py b/airflow/operators/sensors.py
index b561b49..c28e88b 100644
--- a/airflow/operators/sensors.py
+++ b/airflow/operators/sensors.py
@@ -633,8 +633,8 @@ class HttpSensor(BaseSensorOperator):
     :type method: string
     :param endpoint: The relative part of the full url
     :type endpoint: string
-    :param params: The parameters to be added to the GET url
-    :type params: a dictionary of string key/value pairs
+    :param request_params: The parameters to be added to the GET url
+    :type request_params: a dictionary of string key/value pairs
     :param headers: The HTTP headers to be added to the GET request
     :type headers: a dictionary of string key/value pairs
     :param response_check: A check against the 'requests' response object.
@@ -646,21 +646,21 @@ class HttpSensor(BaseSensorOperator):
         depends on the option that's being modified.
     """
 
-    template_fields = ('endpoint', 'params')
+    template_fields = ('endpoint', 'request_params')
 
     @apply_defaults
     def __init__(self,
                  endpoint,
                  http_conn_id='http_default',
                  method='GET',
-                 params=None,
+                 request_params=None,
                  headers=None,
                  response_check=None,
                  extra_options=None, *args, **kwargs):
         super(HttpSensor, self).__init__(*args, **kwargs)
         self.endpoint = endpoint
         self.http_conn_id = http_conn_id
-        self.params = params or {}
+        self.request_params = request_params or {}
         self.headers = headers or {}
         self.extra_options = extra_options or {}
         self.response_check = response_check
@@ -673,7 +673,7 @@ class HttpSensor(BaseSensorOperator):
         logging.info('Poking: ' + self.endpoint)
         try:
             response = self.hook.run(self.endpoint,
-                                     data=self.params,
+                                     data=self.request_params,
                                      headers=self.headers,
                                      extra_options=self.extra_options)
             if self.response_check:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f1ff09d0/tests/core.py
----------------------------------------------------------------------
diff --git a/tests/core.py b/tests/core.py
index 9f43678..d0b9a97 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -2033,7 +2033,7 @@ class HttpOpSensorTest(unittest.TestCase):
             task_id='http_sensor_check',
             http_conn_id='http_default',
             endpoint='/search',
-            params={"client": "ubuntu", "q": "airflow", 'date': '{{ds}}'},
+            request_params={"client": "ubuntu", "q": "airflow", 'date': 
'{{ds}}'},
             headers={},
             response_check=lambda response: (
                 "airbnb/airflow/" + DEFAULT_DATE.strftime('%Y-%m-%d')

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/f1ff09d0/tests/operators/sensors.py
----------------------------------------------------------------------
diff --git a/tests/operators/sensors.py b/tests/operators/sensors.py
index ea1e6cc..242abac 100644
--- a/tests/operators/sensors.py
+++ b/tests/operators/sensors.py
@@ -115,7 +115,7 @@ class HttpSensorTests(unittest.TestCase):
             task_id='http_sensor_poke_exception',
             http_conn_id='http_default',
             endpoint='',
-            params={},
+            request_params={},
             response_check=resp_check,
             poke_interval=5)
         with self.assertRaisesRegexp(AirflowException, 'AirflowException 
raised here!'):
@@ -131,7 +131,7 @@ class HttpSensorTests(unittest.TestCase):
             task_id='http_sensor_head_method',
             http_conn_id='http_default',
             endpoint='',
-            params={},
+            request_params={},
             method='HEAD',
             response_check=resp_check,
             timeout=5,
@@ -173,7 +173,7 @@ class HttpSensorTests(unittest.TestCase):
             task_id='http_sensor_head_method',
             http_conn_id='http_default',
             endpoint='',
-            params={},
+            request_params={},
             method='HEAD',
             response_check=resp_check,
             timeout=5,

Reply via email to