Skip to content

Commit

Permalink
Deferrable mode for Sqs Sensor (#32809)
Browse files Browse the repository at this point in the history
  • Loading branch information
syedahsn committed Aug 4, 2023
1 parent bcfadcf commit 087d14a
Show file tree
Hide file tree
Showing 7 changed files with 428 additions and 51 deletions.
94 changes: 44 additions & 50 deletions airflow/providers/amazon/aws/sensors/sqs.py
Expand Up @@ -18,20 +18,22 @@
"""Reads and then deletes the message from SQS queue."""
from __future__ import annotations

import json
from functools import cached_property
from typing import TYPE_CHECKING, Any, Collection, Literal, Sequence

from deprecated import deprecated
from jsonpath_ng import parse

from airflow.configuration import conf
from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.base_aws import BaseAwsConnection
from airflow.providers.amazon.aws.hooks.sqs import SqsHook
from airflow.providers.amazon.aws.triggers.sqs import SqsSensorTrigger
from airflow.providers.amazon.aws.utils.sqs import process_response
from airflow.sensors.base import BaseSensorOperator

if TYPE_CHECKING:
from airflow.utils.context import Context
from datetime import timedelta


class SqsSensor(BaseSensorOperator):
Expand Down Expand Up @@ -70,6 +72,9 @@ class SqsSensor(BaseSensorOperator):
:param delete_message_on_reception: Default to `True`, the messages are deleted from the queue
as soon as being consumed. Otherwise, the messages remain in the queue after consumption and
should be deleted manually.
:param deferrable: If True, the sensor will operate in deferrable more. This mode requires aiobotocore
module to be installed.
(default: False, but can be overridden in config file by setting default_deferrable to True)
"""

Expand All @@ -88,6 +93,7 @@ def __init__(
message_filtering_match_values: Any = None,
message_filtering_config: Any = None,
delete_message_on_reception: bool = True,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
**kwargs,
):
super().__init__(**kwargs)
Expand All @@ -112,6 +118,34 @@ def __init__(
raise TypeError("message_filtering_match_values must be specified for literal matching")

self.message_filtering_config = message_filtering_config
self.deferrable = deferrable

def execute(self, context: Context) -> Any:
if self.deferrable:
self.defer(
trigger=SqsSensorTrigger(
sqs_queue=self.sqs_queue,
aws_conn_id=self.aws_conn_id,
max_messages=self.max_messages,
num_batches=self.num_batches,
wait_time_seconds=self.wait_time_seconds,
visibility_timeout=self.visibility_timeout,
message_filtering=self.message_filtering,
message_filtering_match_values=self.message_filtering_match_values,
message_filtering_config=self.message_filtering_config,
delete_message_on_reception=self.delete_message_on_reception,
waiter_delay=int(self.poke_interval),
),
method_name="execute_complete",
timeout=timedelta(seconds=self.timeout),
)
else:
super().execute(context=context)

def execute_complete(self, context: Context, event: dict | None = None) -> None:
if event is None or event["status"] != "success":
raise AirflowException(f"Trigger error: event is {event}")
context["ti"].xcom_push(key="messages", value=event["message_batch"])

def poll_sqs(self, sqs_conn: BaseAwsConnection) -> Collection:
"""
Expand All @@ -131,19 +165,7 @@ def poll_sqs(self, sqs_conn: BaseAwsConnection) -> Collection:
receive_message_kwargs["VisibilityTimeout"] = self.visibility_timeout

response = sqs_conn.receive_message(**receive_message_kwargs)

if "Messages" not in response:
return []

messages = response["Messages"]
num_messages = len(messages)
self.log.info("Received %d messages", num_messages)

if num_messages and self.message_filtering:
messages = self.filter_messages(messages)
num_messages = len(messages)
self.log.info("There are %d messages left after filtering", num_messages)
return messages
return response

def poke(self, context: Context):
"""
Expand All @@ -156,7 +178,13 @@ def poke(self, context: Context):

# perform multiple SQS call to retrieve messages in series
for _ in range(self.num_batches):
messages = self.poll_sqs(sqs_conn=self.hook.conn)
response = self.poll_sqs(sqs_conn=self.hook.conn)
messages = process_response(
response,
self.message_filtering,
self.message_filtering_match_values,
self.message_filtering_config,
)

if not len(messages):
continue
Expand Down Expand Up @@ -191,37 +219,3 @@ def get_hook(self) -> SqsHook:
@cached_property
def hook(self) -> SqsHook:
return SqsHook(aws_conn_id=self.aws_conn_id)

def filter_messages(self, messages):
if self.message_filtering == "literal":
return self.filter_messages_literal(messages)
if self.message_filtering == "jsonpath":
return self.filter_messages_jsonpath(messages)
else:
raise NotImplementedError("Override this method to define custom filters")

def filter_messages_literal(self, messages):
filtered_messages = []
for message in messages:
if message["Body"] in self.message_filtering_match_values:
filtered_messages.append(message)
return filtered_messages

def filter_messages_jsonpath(self, messages):
jsonpath_expr = parse(self.message_filtering_config)
filtered_messages = []
for message in messages:
body = message["Body"]
# Body is a string, deserialize to an object and then parse
body = json.loads(body)
results = jsonpath_expr.find(body)
if not results:
continue
if self.message_filtering_match_values is None:
filtered_messages.append(message)
continue
for result in results:
if result.value in self.message_filtering_match_values:
filtered_messages.append(message)
break
return filtered_messages
168 changes: 168 additions & 0 deletions airflow/providers/amazon/aws/triggers/sqs.py
@@ -0,0 +1,168 @@
# 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 asyncio
from typing import Any, AsyncIterator, Collection, Literal

from airflow.exceptions import AirflowException
from airflow.providers.amazon.aws.hooks.base_aws import BaseAwsConnection
from airflow.providers.amazon.aws.hooks.sqs import SqsHook
from airflow.providers.amazon.aws.utils.sqs import process_response
from airflow.triggers.base import BaseTrigger, TriggerEvent


class SqsSensorTrigger(BaseTrigger):
"""
Asynchronously get messages from an Amazon SQS queue and then delete the messages from the queue.
:param sqs_queue: The SQS queue url
:param aws_conn_id: AWS connection id
:param max_messages: The maximum number of messages to retrieve for each poke (templated)
:param num_batches: The number of times the sensor will call the SQS API to receive messages (default: 1)
:param wait_time_seconds: The time in seconds to wait for receiving messages (default: 1 second)
:param visibility_timeout: Visibility timeout, a period of time during which
Amazon SQS prevents other consumers from receiving and processing the message.
:param message_filtering: Specified how received messages should be filtered. Supported options are:
`None` (no filtering, default), `'literal'` (message Body literal match) or `'jsonpath'`
(message Body filtered using a JSONPath expression).
You may add further methods by overriding the relevant class methods.
:param message_filtering_match_values: Optional value/s for the message filter to match on.
For example, with literal matching, if a message body matches any of the specified values
then it is included. For JSONPath matching, the result of the JSONPath expression is used
and may match any of the specified values.
:param message_filtering_config: Additional configuration to pass to the message filter.
For example with JSONPath filtering you can pass a JSONPath expression string here,
such as `'foo[*].baz'`. Messages with a Body which does not match are ignored.
:param delete_message_on_reception: Default to `True`, the messages are deleted from the queue
as soon as being consumed. Otherwise, the messages remain in the queue after consumption and
should be deleted manually.
:param waiter_delay: The time in seconds to wait between calls to the SQS API to receive messages.
"""

def __init__(
self,
sqs_queue: str,
aws_conn_id: str = "aws_default",
max_messages: int = 5,
num_batches: int = 1,
wait_time_seconds: int = 1,
visibility_timeout: int | None = None,
message_filtering: Literal["literal", "jsonpath"] | None = None,
message_filtering_match_values: Any = None,
message_filtering_config: Any = None,
delete_message_on_reception: bool = True,
waiter_delay: int = 60,
):
self.sqs_queue = sqs_queue
self.aws_conn_id = aws_conn_id
self.max_messages = max_messages
self.num_batches = num_batches
self.wait_time_seconds = wait_time_seconds
self.visibility_timeout = visibility_timeout
self.message_filtering = message_filtering
self.delete_message_on_reception = delete_message_on_reception
self.message_filtering_match_values = message_filtering_match_values
self.message_filtering_config = message_filtering_config
self.waiter_delay = waiter_delay

def serialize(self) -> tuple[str, dict[str, Any]]:
return (
self.__class__.__module__ + "." + self.__class__.__qualname__,
{
"sqs_queue": self.sqs_queue,
"aws_conn_id": self.aws_conn_id,
"max_messages": self.max_messages,
"num_batches": self.num_batches,
"wait_time_seconds": self.wait_time_seconds,
"visibility_timeout": self.visibility_timeout,
"message_filtering": self.message_filtering,
"delete_message_on_reception": self.delete_message_on_reception,
"message_filtering_match_values": self.message_filtering_match_values,
"message_filtering_config": self.message_filtering_config,
"waiter_delay": self.waiter_delay,
},
)

@property
def hook(self) -> SqsHook:
return SqsHook(aws_conn_id=self.aws_conn_id)

async def poll_sqs(self, client: BaseAwsConnection) -> Collection:
"""
Asynchronously poll SQS queue to retrieve messages.
:param client: SQS connection
:return: A list of messages retrieved from SQS
"""
self.log.info("SqsSensor checking for message on queue: %s", self.sqs_queue)

receive_message_kwargs = {
"QueueUrl": self.sqs_queue,
"MaxNumberOfMessages": self.max_messages,
"WaitTimeSeconds": self.wait_time_seconds,
}
if self.visibility_timeout is not None:
receive_message_kwargs["VisibilityTimeout"] = self.visibility_timeout

response = await client.receive_message(**receive_message_kwargs)
return response

async def poke(self, client: Any):
message_batch: list[Any] = []
for _ in range(self.num_batches):
self.log.info("starting call to poll sqs")
response = await self.poll_sqs(client=client)
messages = process_response(
response,
self.message_filtering,
self.message_filtering_match_values,
self.message_filtering_config,
)

if not messages:
continue

message_batch.extend(messages)

if self.delete_message_on_reception:
self.log.info("Deleting %d messages", len(messages))

entries = [
{"Id": message["MessageId"], "ReceiptHandle": message["ReceiptHandle"]}
for message in messages
]
response = await client.delete_message_batch(QueueUrl=self.sqs_queue, Entries=entries)

if "Successful" not in response:
raise AirflowException(
f"Delete SQS Messages failed {str(response)} for messages {str(messages)}"
)

return message_batch

async def run(self) -> AsyncIterator[TriggerEvent]:
while True:
# This loop will run indefinitely until the timeout, which is set in the self.defer
# method, is reached.
async with self.hook.async_conn as client:
result = await self.poke(client=client)
if result:
yield TriggerEvent({"status": "success", "message_batch": result})
break
else:
await asyncio.sleep(self.waiter_delay)

0 comments on commit 087d14a

Please sign in to comment.