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/01 09:52:54 UTC

[GitHub] [airflow] phanikumv commented on a diff in pull request #24038: Implement Azure Service Bus Queue Operator's

phanikumv commented on code in PR #24038:
URL: https://github.com/apache/airflow/pull/24038#discussion_r886606774


##########
airflow/providers/microsoft/azure/hooks/base_asb.py:
##########
@@ -0,0 +1,73 @@
+# 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 typing import Any, Dict, Optional
+
+from airflow.hooks.base import BaseHook
+
+
+class BaseAzureServiceBusHook(BaseHook):
+    """
+    BaseAzureServiceBusHook class to session creation and  connection creation. Client ID and
+    Secrete IDs are optional.
+
+    :param azure_service_bus_conn_id: Reference to the
+        :ref:`Azure Service Bus connection<howto/connection:azure_service_bus>`.
+    """
+
+    conn_name_attr = 'azure_service_bus_conn_id'
+    default_conn_name = 'azure_service_bus_default'
+    conn_type = 'azure_service_bus'
+    hook_name = 'Azure ServiceBus'
+
+    @staticmethod
+    def get_connection_form_widgets() -> Dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import StringField
+
+        return {
+            "extra__azure_service_bus__connection_string": StringField(
+                lazy_gettext('Service Bus Connection String'), widget=BS3TextFieldWidget()

Review Comment:
   ```suggestion
                   lazy_gettext('Azure Service Bus Connection String'), widget=BS3TextFieldWidget()
   ```



##########
airflow/providers/microsoft/azure/hooks/base_asb.py:
##########
@@ -0,0 +1,73 @@
+# 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 typing import Any, Dict, Optional
+
+from airflow.hooks.base import BaseHook
+
+
+class BaseAzureServiceBusHook(BaseHook):
+    """
+    BaseAzureServiceBusHook class to session creation and  connection creation. Client ID and

Review Comment:
   ```suggestion
       BaseAzureServiceBusHook class to create session and create connection. Client ID and
   ```



##########
airflow/providers/microsoft/azure/operators/azure_service_bus_queue.py:
##########
@@ -0,0 +1,184 @@
+# 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 typing import TYPE_CHECKING, List, Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.microsoft.azure.hooks.asb_admin_client import AzureServiceBusAdminClientHook
+from airflow.providers.microsoft.azure.hooks.asb_message import ServiceBusMessageHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class AzureServiceBusCreateQueueOperator(BaseOperator):
+    """
+    Creates a Azure ServiceBus queue under a ServiceBus Namespace by using ServiceBusAdministrationClient
+
+    :param queue_name: The name of the queue. should be unique.
+    :param azure_service_bus_conn_id: Reference to the
+        :ref:`Azure Service Bus connection<howto/connection:azure_service_bus>`.
+    """
+
+    template_fields: Sequence[str] = ("queue_name",)
+    ui_color = "#e4f0e8"
+
+    def __init__(
+        self,
+        *,
+        queue_name: str,
+        max_delivery_count: int = 10,
+        dead_lettering_on_message_expiration: bool = True,
+        enable_batched_operations: bool = True,
+        azure_service_bus_conn_id: str = 'azure_service_bus_default',
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.queue_name = queue_name
+        self.max_delivery_count = max_delivery_count
+        self.dead_lettering_on_message_expiration = dead_lettering_on_message_expiration
+        self.enable_batched_operations = enable_batched_operations
+        self.azure_service_bus_conn_id = azure_service_bus_conn_id
+
+    def execute(self, context: "Context") -> None:
+        """Creates Queue in Service Bus namespace, by connecting to Service Bus Admin client"""
+        # Create the hook testing

Review Comment:
   ```suggestion
   ```



##########
tests/providers/microsoft/azure/hooks/test_asb_admin_client.py:
##########
@@ -0,0 +1,104 @@
+# 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.
+
+import json
+from unittest import mock
+
+import pytest
+from azure.servicebus.management import ServiceBusAdministrationClient
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.providers.microsoft.azure.hooks.asb_admin_client import AzureServiceBusAdminClientHook
+
+
+class TestAzureServiceBusAdminClientHook:
+    def setup_class(self) -> None:
+        self.queue_name: str = "test_queue"
+        self.conn_id: str = 'azure_service_bus_default'
+        self.connection_string = (
+            "Endpoint=sb://test-service-bus-provider.servicebus.windows.net/;"
+            "SharedAccessKeyName=Test;SharedAccessKey=1234566acbc"
+        )
+        self.client_id = "test_client_id"
+        self.secret_key = "test_client_secret"
+        self.mock_conn = Connection(
+            conn_id='azure_service_bus_default',
+            conn_type='azure_service_bus',
+            login=self.client_id,
+            password=self.secret_key,
+            extra=json.dumps({"connection_string": self.connection_string}),
+        )
+
+    @mock.patch(
+        "airflow.providers.microsoft.azure.hooks.asb_admin_client."
+        "AzureServiceBusAdminClientHook.get_connection"
+    )
+    def test_get_conn(self, mock_connection):
+        mock_connection.return_value = self.mock_conn
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        assert isinstance(hook.get_conn(), ServiceBusAdministrationClient)
+
+    @mock.patch('azure.servicebus.management.QueueProperties')
+    @mock.patch(
+        'airflow.providers.microsoft.azure.hooks.asb_admin_client.AzureServiceBusAdminClientHook.get_conn'
+    )
+    def test_create_queue(self, mock_sb_admin_client, mock_queue_properties):
+        """
+        Test `create_queue` hook function with mocking connection, queue properties value and
+        the azure service bus `create_queue` function
+        """
+        mock_queue_properties.name = self.queue_name
+        mock_sb_admin_client.return_value.__enter__.return_value.create_queue.return_value = (
+            mock_queue_properties
+        )
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        response = hook.create_queue(self.queue_name)
+        assert response == mock_queue_properties
+
+    @mock.patch('airflow.providers.microsoft.azure.hooks.asb_admin_client.ServiceBusAdministrationClient')
+    def test_create_queue_exception(self, mock_sb_admin_client):
+        """
+        Test `create_queue` functionality to raise AirflowException
+        by passing queue name as None and pytest raise Airflow Exception

Review Comment:
   ```suggestion
           by passing queue name as None and checking whether AirflowException is raised
   ```



##########
tests/providers/microsoft/azure/hooks/test_asb_admin_client.py:
##########
@@ -0,0 +1,104 @@
+# 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.
+
+import json
+from unittest import mock
+
+import pytest
+from azure.servicebus.management import ServiceBusAdministrationClient
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.providers.microsoft.azure.hooks.asb_admin_client import AzureServiceBusAdminClientHook
+
+
+class TestAzureServiceBusAdminClientHook:
+    def setup_class(self) -> None:
+        self.queue_name: str = "test_queue"
+        self.conn_id: str = 'azure_service_bus_default'
+        self.connection_string = (
+            "Endpoint=sb://test-service-bus-provider.servicebus.windows.net/;"
+            "SharedAccessKeyName=Test;SharedAccessKey=1234566acbc"
+        )
+        self.client_id = "test_client_id"
+        self.secret_key = "test_client_secret"
+        self.mock_conn = Connection(
+            conn_id='azure_service_bus_default',
+            conn_type='azure_service_bus',
+            login=self.client_id,
+            password=self.secret_key,
+            extra=json.dumps({"connection_string": self.connection_string}),
+        )
+
+    @mock.patch(
+        "airflow.providers.microsoft.azure.hooks.asb_admin_client."
+        "AzureServiceBusAdminClientHook.get_connection"
+    )
+    def test_get_conn(self, mock_connection):
+        mock_connection.return_value = self.mock_conn
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        assert isinstance(hook.get_conn(), ServiceBusAdministrationClient)
+
+    @mock.patch('azure.servicebus.management.QueueProperties')
+    @mock.patch(
+        'airflow.providers.microsoft.azure.hooks.asb_admin_client.AzureServiceBusAdminClientHook.get_conn'
+    )
+    def test_create_queue(self, mock_sb_admin_client, mock_queue_properties):
+        """
+        Test `create_queue` hook function with mocking connection, queue properties value and
+        the azure service bus `create_queue` function
+        """
+        mock_queue_properties.name = self.queue_name
+        mock_sb_admin_client.return_value.__enter__.return_value.create_queue.return_value = (
+            mock_queue_properties
+        )
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        response = hook.create_queue(self.queue_name)
+        assert response == mock_queue_properties
+
+    @mock.patch('airflow.providers.microsoft.azure.hooks.asb_admin_client.ServiceBusAdministrationClient')
+    def test_create_queue_exception(self, mock_sb_admin_client):
+        """
+        Test `create_queue` functionality to raise AirflowException
+        by passing queue name as None and pytest raise Airflow Exception
+        """
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        with pytest.raises(AirflowException):
+            hook.create_queue(None)
+
+    @mock.patch(
+        'airflow.providers.microsoft.azure.hooks.asb_admin_client.AzureServiceBusAdminClientHook.get_conn'
+    )
+    def test_delete_queue(self, mock_sb_admin_client):
+        """
+        Test Delete queue functionality by passing queue name, assert the function with values,
+        mock the azure service bus function  `delete_queue`
+        """
+        hook = AzureServiceBusAdminClientHook(azure_service_bus_conn_id=self.conn_id)
+        hook.delete_queue(self.queue_name)
+        expected_calls = [mock.call().__enter__().delete_queue(self.queue_name)]
+        mock_sb_admin_client.assert_has_calls(expected_calls)
+
+    @mock.patch('airflow.providers.microsoft.azure.hooks.asb_admin_client.ServiceBusAdministrationClient')
+    def test_delete_queue_exception(self, mock_sb_admin_client):
+        """
+        Test `delete_queue` functionality to raise AirflowException,
+         by passing queue name as None and pytest raise Airflow Exception

Review Comment:
   ```suggestion
            by passing queue name as None and checking whether AirflowException is raised
   ```



##########
tests/providers/microsoft/azure/operators/test_azure_service_queue.py:
##########
@@ -0,0 +1,234 @@
+# 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.
+

Review Comment:
   Name the file as tests/providers/microsoft/azure/operators/test_azure_service_bus_queue.py to be consistent with operator file name



##########
docs/apache-airflow-providers-microsoft-azure/connections/azure_service_bus.rst:
##########
@@ -0,0 +1,64 @@
+.. 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.
+
+
+
+.. _howto/connection:azure_service_bus:
+
+Microsoft Azure Service Bus
+=======================================
+
+The Microsoft Azure Service Bus connection type enables the Azure Service Bus Integrations.

Review Comment:
   ```suggestion
   The Microsoft Azure Service Bus connection type enables the Azure Service Bus integration.
   ```



##########
tests/providers/microsoft/azure/hooks/test_asb_message.py:
##########
@@ -0,0 +1,157 @@
+# 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.
+
+import json
+from unittest import mock
+
+import pytest
+from azure.servicebus import ServiceBusClient, ServiceBusMessage, ServiceBusMessageBatch
+
+from airflow import AirflowException
+from airflow.models import Connection
+from airflow.providers.microsoft.azure.hooks.asb_message import ServiceBusMessageHook
+
+
+class TestServiceBusMessageHook:
+    def setup_class(self) -> None:
+        self.queue_name: str = "test_queue"
+        self.conn_id: str = 'azure_service_bus_default'
+        self.connection_string = (
+            "Endpoint=sb://test-service-bus-provider.servicebus.windows.net/;"
+            "SharedAccessKeyName=Test;SharedAccessKey=1234566acbc"
+        )
+        self.client_id = "test_client_id"
+        self.secret_key = "test_client_secret"
+        self.conn = Connection(
+            conn_id='azure_service_bus_default',
+            conn_type='azure_service_bus',
+            login=self.client_id,
+            password=self.secret_key,
+            extra=json.dumps({'connection_string': self.connection_string}),
+        )
+
+    @mock.patch("airflow.providers.microsoft.azure.hooks.asb_message.ServiceBusMessageHook.get_connection")
+    def test_get_service_bus_message_conn(self, mock_connection):
+        mock_connection.return_value = self.conn
+        hook = ServiceBusMessageHook(azure_service_bus_conn_id=self.conn_id)
+        assert isinstance(hook.get_conn(), ServiceBusClient)
+
+    @mock.patch("airflow.providers.microsoft.azure.hooks.asb_message.ServiceBusMessageHook.get_connection")
+    def test_get_conn_value_error(self, mock_connection):
+        mock_connection.return_value = Connection(

Review Comment:
   Please add docstring to describe what you are trying to test here.



##########
airflow/providers/microsoft/azure/hooks/base_asb.py:
##########
@@ -0,0 +1,73 @@
+# 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 typing import Any, Dict, Optional
+
+from airflow.hooks.base import BaseHook
+
+
+class BaseAzureServiceBusHook(BaseHook):
+    """
+    BaseAzureServiceBusHook class to session creation and  connection creation. Client ID and
+    Secrete IDs are optional.
+
+    :param azure_service_bus_conn_id: Reference to the
+        :ref:`Azure Service Bus connection<howto/connection:azure_service_bus>`.
+    """
+
+    conn_name_attr = 'azure_service_bus_conn_id'
+    default_conn_name = 'azure_service_bus_default'
+    conn_type = 'azure_service_bus'
+    hook_name = 'Azure ServiceBus'
+
+    @staticmethod
+    def get_connection_form_widgets() -> Dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import StringField
+
+        return {
+            "extra__azure_service_bus__connection_string": StringField(
+                lazy_gettext('Service Bus Connection String'), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ['schema', 'port', 'host', 'extra'],
+            "relabeling": {
+                'login': 'Client ID',
+                'password': 'Secret',
+            },
+            "placeholders": {
+                'login': 'Client ID (Optional)',
+                'password': 'Client Secret (Optional)',
+                'extra__azure_service_bus__connection_string': 'Service Bus Connection String',

Review Comment:
   ```suggestion
                   'extra__azure_service_bus__connection_string': 'Azure Service Bus Connection String',
   ```



-- 
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