Repository: incubator-airflow Updated Branches: refs/heads/master cac49e6ea -> fd381a11e
[AIRFLOW-645] Support HTTPS connections in HttpHook Consider the connection schema when building the URL in HttpHook. This allows making HTTPS requests. If a schema is not specified in the connection, default to HTTP. If the connection host contains a schema, use it instead to maintain backwards compatibility. Closes #2311 from johnzeringue/AIRFLOW-645 Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/fd381a11 Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/fd381a11 Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/fd381a11 Branch: refs/heads/master Commit: fd381a11e98b60e1ff3d75aea07fbc36d0bc5d38 Parents: cac49e6 Author: John Zeringue <[email protected]> Authored: Sun May 21 20:28:19 2017 +0200 Committer: Bolke de Bruin <[email protected]> Committed: Sun May 21 20:28:19 2017 +0200 ---------------------------------------------------------------------- airflow/hooks/http_hook.py | 10 ++++++--- tests/core.py | 49 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 56 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/fd381a11/airflow/hooks/http_hook.py ---------------------------------------------------------------------- diff --git a/airflow/hooks/http_hook.py b/airflow/hooks/http_hook.py index 041328f..a144143 100644 --- a/airflow/hooks/http_hook.py +++ b/airflow/hooks/http_hook.py @@ -37,9 +37,13 @@ class HttpHook(BaseHook): """ conn = self.get_connection(self.http_conn_id) session = requests.Session() - self.base_url = conn.host - if not self.base_url.startswith('http'): - self.base_url = 'http://' + self.base_url + + if "://" in conn.host: + self.base_url = conn.host + else: + # schema defaults to HTTP + schema = conn.schema if conn.schema else "http" + self.base_url = schema + "://" + conn.host if conn.port: self.base_url = self.base_url + ":" + str(conn.port) + "/" http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/fd381a11/tests/core.py ---------------------------------------------------------------------- diff --git a/tests/core.py b/tests/core.py index a4757a7..9f43678 100644 --- a/tests/core.py +++ b/tests/core.py @@ -2278,6 +2278,55 @@ class HDFSHookTest(unittest.TestCase): try: + from airflow.hooks.http_hook import HttpHook +except ImportError: + HttpHook = None + + [email protected](HttpHook is None, + "Skipping test because HttpHook is not installed") +class HttpHookTest(unittest.TestCase): + def setUp(self): + configuration.load_test_config() + + @mock.patch('airflow.hooks.http_hook.HttpHook.get_connection') + def test_http_connection(self, mock_get_connection): + c = models.Connection(conn_id='http_default', conn_type='http', + host='localhost', schema='http') + mock_get_connection.return_value = c + hook = HttpHook() + hook.get_conn({}) + self.assertEqual(hook.base_url, 'http://localhost') + + @mock.patch('airflow.hooks.http_hook.HttpHook.get_connection') + def test_https_connection(self, mock_get_connection): + c = models.Connection(conn_id='http_default', conn_type='http', + host='localhost', schema='https') + mock_get_connection.return_value = c + hook = HttpHook() + hook.get_conn({}) + self.assertEqual(hook.base_url, 'https://localhost') + + @mock.patch('airflow.hooks.http_hook.HttpHook.get_connection') + def test_host_encoded_http_connection(self, mock_get_connection): + c = models.Connection(conn_id='http_default', conn_type='http', + host='http://localhost') + mock_get_connection.return_value = c + hook = HttpHook() + hook.get_conn({}) + self.assertEqual(hook.base_url, 'http://localhost') + + @mock.patch('airflow.hooks.http_hook.HttpHook.get_connection') + def test_host_encoded_https_connection(self, mock_get_connection): + c = models.Connection(conn_id='http_default', conn_type='http', + host='https://localhost') + mock_get_connection.return_value = c + hook = HttpHook() + hook.get_conn({}) + self.assertEqual(hook.base_url, 'https://localhost') + + +try: from airflow.hooks.S3_hook import S3Hook except ImportError: S3Hook = None
