Skip to content

Commit

Permalink
Add deferrable mode to SimpleHttpOperator
Browse files Browse the repository at this point in the history
  • Loading branch information
tnk-ysk committed Jul 8, 2023
1 parent 0f73647 commit 9278c6f
Show file tree
Hide file tree
Showing 6 changed files with 346 additions and 14 deletions.
63 changes: 50 additions & 13 deletions airflow/providers/http/operators/http.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,15 @@
from __future__ import annotations

from typing import TYPE_CHECKING, Any, Callable, Sequence

from requests import Response
from requests.auth import AuthBase
import pickle
import base64

from airflow.exceptions import AirflowException
from airflow.models import BaseOperator
from airflow.providers.http.hooks.http import HttpHook
from airflow.providers.http.triggers.http import HttpTrigger

if TYPE_CHECKING:
from airflow.utils.context import Context
Expand Down Expand Up @@ -89,6 +92,7 @@ def __init__(
tcp_keep_alive_idle: int = 120,
tcp_keep_alive_count: int = 20,
tcp_keep_alive_interval: int = 30,
deferrable: bool = False,
**kwargs: Any,
) -> None:
super().__init__(**kwargs)
Expand All @@ -106,23 +110,44 @@ def __init__(
self.tcp_keep_alive_idle = tcp_keep_alive_idle
self.tcp_keep_alive_count = tcp_keep_alive_count
self.tcp_keep_alive_interval = tcp_keep_alive_interval
self.deferrable = deferrable

def execute(self, context: Context) -> Any:
from airflow.utils.operator_helpers import determine_kwargs
if self.deferrable:
self.defer(
trigger=HttpTrigger(
http_conn_id=self.http_conn_id,
auth_type=self.auth_type,
method=self.method,
endpoint=self.endpoint,
headers=self.headers,
data=self.data,
extra_options=self.extra_options,
),
method_name="execute_complete",
)
else:
http = HttpHook(
self.method,
http_conn_id=self.http_conn_id,
auth_type=self.auth_type,
tcp_keep_alive=self.tcp_keep_alive,
tcp_keep_alive_idle=self.tcp_keep_alive_idle,
tcp_keep_alive_count=self.tcp_keep_alive_count,
tcp_keep_alive_interval=self.tcp_keep_alive_interval,
)

self.log.info("Calling HTTP method")

http = HttpHook(
self.method,
http_conn_id=self.http_conn_id,
auth_type=self.auth_type,
tcp_keep_alive=self.tcp_keep_alive,
tcp_keep_alive_idle=self.tcp_keep_alive_idle,
tcp_keep_alive_count=self.tcp_keep_alive_count,
tcp_keep_alive_interval=self.tcp_keep_alive_interval,
)
response = http.run(self.endpoint, self.data, self.headers, self.extra_options)
return self.process_response(context=context, response=response)

self.log.info("Calling HTTP method")
def process_response(self, context: Context, response: Response) -> str:
"""
Process the response.
"""
from airflow.utils.operator_helpers import determine_kwargs

response = http.run(self.endpoint, self.data, self.headers, self.extra_options)
if self.log_response:
self.log.info(response.text)
if self.response_check:
Expand All @@ -133,3 +158,15 @@ def execute(self, context: Context) -> Any:
kwargs = determine_kwargs(self.response_filter, [response], context)
return self.response_filter(response, **kwargs)
return response.text

def execute_complete(self, context: Context, event: dict):
"""
Callback for when the trigger fires - returns immediately.
Relies on trigger to throw an exception, otherwise it assumes execution was successful.
"""
if event["status"] == "success":
response = pickle.loads(base64.standard_b64decode(event["response"]))
return self.process_response(context=context, response=response)
else:
raise AirflowException(f"Unexpected error in the operation: {event['message']}")
Empty file.
126 changes: 126 additions & 0 deletions airflow/providers/http/triggers/http.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,126 @@
# 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 __future__ import annotations
import traceback
from typing import Any, AsyncIterator
import pickle
import base64
import requests
from requests.structures import CaseInsensitiveDict

from airflow.providers.http.hooks.http import HttpAsyncHook
from airflow.triggers.base import BaseTrigger, TriggerEvent

from aiohttp.client_reqrep import ClientResponse


class HttpTrigger(BaseTrigger):
"""
HttpTrigger run on the trigger worker.
:param http_conn_id: http connection id that has the base
API url i.e https://www.google.com/ and optional authentication credentials. Default
headers can also be specified in the Extra field in json format.
:param auth_type: The auth type for the service
:param method: the API method to be called
:param endpoint: Endpoint to be called, i.e. ``resource/v1/query?``.
:param headers: Additional headers to be passed through as a dict.
:param data: Payload to be uploaded or request parameters.
:param extra_options: Additional kwargs to pass when creating a request.
For example, ``run(json=obj)`` is passed as
``aiohttp.ClientSession().get(json=obj)``.
2XX or 3XX status codes
"""

def __init__(
self,
http_conn_id: str = "http_default",
auth_type: Any = None,
method: str = "POST",
endpoint: str | None = None,
headers: dict[str, str] | None = None,
data: Any = None,
extra_options: dict[str, Any] | None = None,
):
super().__init__()
self.http_conn_id = http_conn_id
self.method = method
self.auth_type = auth_type
self.endpoint = endpoint
self.headers = headers
self.data = data
self.extra_options = extra_options

def serialize(self) -> tuple[str, dict[str, Any]]:
"""Serializes HttpTrigger arguments and classpath."""
return (
"airflow.providers.http.triggers.http.HttpTrigger",
{
"http_conn_id": self.http_conn_id,
"method": self.method,
"auth_type": self.auth_type,
"endpoint": self.endpoint,
"headers": self.headers,
"data": self.data,
"extra_options": self.extra_options,
},
)

async def run(self) -> AsyncIterator[TriggerEvent]:
"""
Makes a series of asynchronous http calls via an http hook. It yields a Trigger if
response is a 200 and run_state is successful, will retry the call up to the retry limit
if the error is 'retryable', otherwise it throws an exception.
"""
hook = HttpAsyncHook(
method=self.method,
http_conn_id=self.http_conn_id,
auth_type=self.auth_type,
)
try:
client_response = await hook.run(
endpoint=self.endpoint,
data=self.data,
headers=self.headers,
extra_options=self.extra_options,
)
response = await self._convert_response(client_response)
yield TriggerEvent(
{
"status": "success",
"response": base64.standard_b64encode(pickle.dumps(response)).decode("ascii"),
}
)
except Exception as e:
yield TriggerEvent({"status": "error", "message": str(e)})
# yield TriggerEvent({"status": "error", "message": str(traceback.format_exc())})

@staticmethod
async def _convert_response(client_response: ClientResponse) -> requests.Response:
"""
Convert aiohttp.client_reqrep.ClientResponse to requests.Response.
"""
response = requests.Response()
response._content = await client_response.read()
response.status_code = client_response.status
response.headers = CaseInsensitiveDict(client_response.headers)
response.url = client_response.url
response.history = client_response.history
response.encoding = client_response.get_encoding()
response.reason = client_response.reason
response.cookies = client_response.cookies
return response
31 changes: 30 additions & 1 deletion tests/providers/http/operators/test_http.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,16 @@
# under the License.
from __future__ import annotations

import base64
import pickle
from unittest import mock

import pytest
from requests import Response

from airflow.exceptions import AirflowException
from airflow.exceptions import AirflowException, TaskDeferred
from airflow.providers.http.operators.http import SimpleHttpOperator
from airflow.providers.http.triggers.http import HttpTrigger


@mock.patch.dict("os.environ", AIRFLOW_CONN_HTTP_EXAMPLE="http://www.example.com")
Expand Down Expand Up @@ -81,3 +85,28 @@ def test_filters_response(self, requests_mock):
)
result = operator.execute({})
assert result == {"value": 5}

def test_async_defer_successfully(self, requests_mock):
operator = SimpleHttpOperator(
task_id="test_HTTP_op",
deferrable=True,
)
with pytest.raises(TaskDeferred) as exc:
operator.execute({})
assert isinstance(exc.value.trigger, HttpTrigger), "Trigger is not a HttpTrigger"

def test_async_execute_successfully(self, requests_mock):
operator = SimpleHttpOperator(
task_id="test_HTTP_op",
deferrable=True,
)
response = Response()
response._content = "content".encode("utf-8")
result = operator.execute_complete(
context={},
event={
"status": "success",
"response": base64.standard_b64encode(pickle.dumps(response)).decode("ascii"),
},
)
assert "content" == result
Empty file.
Loading

0 comments on commit 9278c6f

Please sign in to comment.