You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/06/21 18:49:59 UTC

[GitHub] [airflow] eladkal commented on a diff in pull request #24554: Add SqsBatchSensor

eladkal commented on code in PR #24554:
URL: https://github.com/apache/airflow/pull/24554#discussion_r902956729


##########
airflow/providers/amazon/aws/sensors/sqs.py:
##########
@@ -215,3 +229,68 @@ def __init__(self, *args, **kwargs):
             stacklevel=2,
         )
         super().__init__(*args, **kwargs)
+
+
+class SqsBatchSensor(SqsSensor):
+    """
+    Get messages from an Amazon SQS queue in batches and then delete the retrieved messages from the queue.
+    If deletion of messages fails an AirflowException is thrown. Otherwise, all messages
+    are pushed through XCom with the key ``messages``.
+
+    This sensor is identical to SqsSensor, except that SqsSensor performs one and only one SQS call
+    per poke, which limits the result to a maximum of 10 messages, while SqsBatchSensor performs multiple
+    SQS API calls per poke and combines the results into one list.
+
+    .. seealso::
+        For more information on how to use this sensor, take a look at the guide:
+        :ref:`howto/sensor:SqsBatchSensor`
+
+    :param num_batches: The number of times the sensor will call the SQS API to receive messages (default: 1)
+    """
+
+    def __init__(
+        self,
+        *,
+        num_batches: int = 1,
+        **kwargs,
+    ):
+        super().__init__(**kwargs)
+        self.num_batches = num_batches
+
+    def poke(self, context: 'Context'):
+        """
+        Check subscribed queue for messages and write them to xcom with the ``messages`` key.
+
+        :param context: the context object
+        :return: ``True`` if message is available or ``False``
+        """
+        sqs_conn = self.get_hook().get_conn()
+        message_batch: List[Any] = []
+        # perform multiple SQS call to retrieve messages in series
+        for _ in range(self.num_batches):
+            messages = self.poll_sqs(sqs_conn=sqs_conn)
+
+            if not len(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 = sqs_conn.delete_message_batch(QueueUrl=self.sqs_queue, Entries=entries)
+
+                if 'Successful' not in response:
+                    raise AirflowException(
+                        'Delete SQS Messages failed ' + str(response) + ' for messages ' + str(messages)
+                    )
+        if not len(message_batch):
+            return False

Review Comment:
   I understand that this is a batch version of `SqsSensor` so the code here is expected. However, this feels too much for a sensor. To my perspective (and it's OK to disagree) sensors are suppose to be very simple e.g wait for condition to met. Is file present? is status changed? etc...
   Here it feels like the sensor is also taken the role of "regular operator". Not only that it waits for message(s) it also retrieve all of them and remove them.
   That is : check for messages, read them, delete them - one might argue that this sensor is doing the job of 3 operators.
   
   Wouldn't it be simpler if sensor would just be sensor? e.g wait for messages in queue. if message present then just return True. let downstream task (operator) to handle the retrieval of message(s) and the delete.
   
   Assuming that sensor would just be sensor then sensor batch version wouldn't be needed to begin with.
   
   @ferruzzi @o-nikolas @LaPetiteSouris WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org