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/08/17 17:51:02 UTC

[GitHub] [airflow] gitstart-airflow opened a new pull request, #25769: Add support for MS Teams webhook operator

gitstart-airflow opened a new pull request, #25769:
URL: https://github.com/apache/airflow/pull/25769

   closes: https://github.com/apache/airflow/issues/25504
   related: https://github.com/apache/airflow/issues/25504
   
   This PR adds support for MS Teams operator and hook for communicating with MS Teams channels


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


[GitHub] [airflow] eladkal commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r949924709


##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,132 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            extra = conn.extra_dejson
+            web_token = extra.get('webhook_token', '')
+            return web_token
+        else:
+            raise AirflowException('Cannot get token: No valid Teams webhook token nor conn_id supplied')
+
+    def _build_teams_message(self) -> str:
+        """
+        Construct Teams connector message. All the relevant parameters are combined to a valid
+        Teams json message.
+
+        :return: JSON formated MS Teams connector message to send
+        :rtype: str
+        """
+        card = {
+            "themeColor": self.theme_color,
+            "summary": self.message,
+            "sections": [{
+                "activityTitle": self.message,
+                "activitySubtitle": self.subtitle,
+                "activityImage": self.icon_url,
+                "facts": self.facts,
+                "potentialAction": [{
+                    "@context": "http://schema.org",
+                    "@type": "OpenUri",
+                    "name": self.action_button_name,
+                    "targets": [{
+                        "os": "default",
+                        "uri": self.action_button_url
+                    }]
+                }]}
+            ]
+        }
+        return json.dumps(card)
+
+    def execute(self) -> None:

Review Comment:
   OK so lets just change the `execute` to `send`



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


[GitHub] [airflow] josh-fell commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r952026091


##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,133 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,

Review Comment:
   Would you mind adding type annotations here please?



##########
tests/providers/microsoft/teams/hooks/test_teams_webhook.py:
##########
@@ -0,0 +1,90 @@
+#
+# 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
+import unittest
+
+from airflow.models import Connection
+from airflow.providers.microsoft.teams.hooks.teams_webhook import TeamsWebhookHook
+from airflow.utils import db
+
+
+class TestTeamsWebhookHook(unittest.TestCase):

Review Comment:
   Can you write this with `pytest` instead? For net-new unit tests, the preference is to use `pytest` over `unittest`.



##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,133 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            extra = conn.extra_dejson
+            web_token = extra.get('webhook_token', '')

Review Comment:
   Instead of accessing the `webhook_token` via Extras, WDYT about using `conn.password` instead? Extras have historically gotten a little messy.



##########
tests/providers/microsoft/teams/operators/test_teams_webhook.py:
##########
@@ -0,0 +1,62 @@
+#
+# 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 unittest
+
+from airflow.models.dag import DAG
+from airflow.providers.microsoft.teams.operators.teams_webhook import TeamsWebhookOperator
+from airflow.utils import timezone
+
+DEFAULT_DATE = timezone.datetime(2017, 1, 1)
+
+
+class TestTeamsWebhookOperator(unittest.TestCase):

Review Comment:
   Same here.



##########
airflow/providers/microsoft/teams/operators/teams_webhook.py:
##########
@@ -0,0 +1,69 @@
+#
+# 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, Optional
+
+from airflow.providers.http.operators.http import SimpleHttpOperator
+from airflow.providers.microsoft.teams.hooks.teams_webhook import TeamsWebhookHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TeamsWebhookOperator(SimpleHttpOperator):
+    def __init__(
+        self,
+        http_conn_id: str,
+        webhook_token: Optional[str] = None,
+        message: str = "",
+        subtitle: Optional[str] = None,
+        theme_color: Optional[str] = None,
+        facts: Optional[list] = None,
+        action_button_name: Optional[str] = None,
+        action_button_url: Optional[str] = None,
+        icon_url: Optional[str] = None,
+        proxy: Optional[str] = None,
+        **kwargs,
+    ):
+        super().__init__(endpoint=webhook_token, **kwargs)
+        self.http_conn_id = http_conn_id
+        self.webhook_token = webhook_token
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+        self.hook: Optional[TeamsWebhookHook] = None
+
+    def execute(self, context: 'Context') -> None:
+        self.hook = TeamsWebhookHook(
+            self.http_conn_id,
+            self.webhook_token,
+            self.message,
+            self.subtitle,
+            self.theme_color,
+            self.facts,
+            self.action_button_name,
+            self.action_button_url,
+            self.icon_url,
+            self.proxy,
+        )
+        self.hook.send()

Review Comment:
   ```suggestion
   
       def execute(self, context: 'Context') -> None:
           hook = TeamsWebhookHook(
               self.http_conn_id,
               self.webhook_token,
               self.message,
               self.subtitle,
               self.theme_color,
               self.facts,
               self.action_button_name,
               self.action_button_url,
               self.icon_url,
               self.proxy,
           )
           hook.send()
   ```
   Since this object is only used in one method, IMO it doesn't make sense for it to be an instance attr.



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


[GitHub] [airflow] spatocode commented on pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
spatocode commented on PR #25769:
URL: https://github.com/apache/airflow/pull/25769#issuecomment-1247923979

   @eladkal There are two remaining failing breeze test. I would need help here


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


[GitHub] [airflow] potiuk commented on pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25769:
URL: https://github.com/apache/airflow/pull/25769#issuecomment-1250608255

   Hey @gitstart-airflow 
   
   1) You need to rebase the PR to latest main version
   2) Install pre-commit and make sure that all static checks are passing after rebase - fix them if they are failing in your PR 


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


[GitHub] [airflow] josh-fell commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r963676736


##########
airflow/providers/microsoft/teams/operators/teams_webhook.py:
##########
@@ -0,0 +1,69 @@
+#
+# 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, Optional
+
+from airflow.providers.http.operators.http import SimpleHttpOperator
+from airflow.providers.microsoft.teams.hooks.teams_webhook import TeamsWebhookHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TeamsWebhookOperator(SimpleHttpOperator):
+    def __init__(
+        self,
+        http_conn_id: str,
+        webhook_token: Optional[str] = None,
+        message: str = "",
+        subtitle: Optional[str] = None,
+        theme_color: Optional[str] = None,
+        facts: Optional[list] = None,
+        action_button_name: Optional[str] = None,
+        action_button_url: Optional[str] = None,
+        icon_url: Optional[str] = None,
+        proxy: Optional[str] = None,
+        **kwargs,
+    ):
+        super().__init__(endpoint=webhook_token, **kwargs)
+        self.http_conn_id = http_conn_id
+        self.webhook_token = webhook_token
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+        self.hook: Optional[TeamsWebhookHook] = None

Review Comment:
   ```suggestion
   ```
   Not necessary, or used, as an instance attr.



##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,132 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id: str = None,
+        webhook_token: Optional[str] = None,
+        message: Optional[str] = None,
+        subtitle: Optional[str] = None,
+        theme_color: Optional[str] = None,
+        facts: Optional[list] = None,
+        action_button_name: Optional[str] = None,
+        action_button_url: Optional[str] = None,
+        icon_url: Optional[str] = None,
+        proxy: Optional[str] = None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            password = getattr(conn, 'password', None)
+            return password

Review Comment:
   Should there be a check here that `password` _actually_ contains a value?



##########
airflow/providers/microsoft/teams/operators/teams_webhook.py:
##########
@@ -0,0 +1,69 @@
+#
+# 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, Optional
+
+from airflow.providers.http.operators.http import SimpleHttpOperator
+from airflow.providers.microsoft.teams.hooks.teams_webhook import TeamsWebhookHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TeamsWebhookOperator(SimpleHttpOperator):

Review Comment:
   Would it be useful for any of the operator attrs to be added in `template_fields` or are you thinking the ones defined in `SimpleHttpOperator` are sufficient? My initial reaction is `message` may be useful to allow dynamic content via Jinja templating. 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


[GitHub] [airflow] gitstart-airflow closed pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
gitstart-airflow closed pull request #25769: Add support for MS Teams webhook operator
URL: https://github.com/apache/airflow/pull/25769


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


[GitHub] [airflow] gitstart-airflow commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
gitstart-airflow commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r955235584


##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,133 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            extra = conn.extra_dejson
+            web_token = extra.get('webhook_token', '')

Review Comment:
   Yes, you're right. Webtokens in extras is also depreciated.



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


[GitHub] [airflow] spatocode commented on pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
spatocode commented on PR #25769:
URL: https://github.com/apache/airflow/pull/25769#issuecomment-1231378502

   Awaiting review @josh-fell @eladkal @mik-laj @potiuk 


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


[GitHub] [airflow] spatocode commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
spatocode commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r965101223


##########
airflow/providers/microsoft/teams/operators/teams_webhook.py:
##########
@@ -0,0 +1,69 @@
+#
+# 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, Optional
+
+from airflow.providers.http.operators.http import SimpleHttpOperator
+from airflow.providers.microsoft.teams.hooks.teams_webhook import TeamsWebhookHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TeamsWebhookOperator(SimpleHttpOperator):

Review Comment:
   Good point. I've added couple of template_fields



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


[GitHub] [airflow] eladkal commented on pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
eladkal commented on PR #25769:
URL: https://github.com/apache/airflow/pull/25769#issuecomment-1234671221

   @spatocode tests are failing. you'll need to address it first


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


[GitHub] [airflow] spatocode commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
spatocode commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r949365495


##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,132 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            extra = conn.extra_dejson
+            web_token = extra.get('webhook_token', '')
+            return web_token
+        else:
+            raise AirflowException('Cannot get token: No valid Teams webhook token nor conn_id supplied')
+
+    def _build_teams_message(self) -> str:
+        """
+        Construct Teams connector message. All the relevant parameters are combined to a valid
+        Teams json message.
+
+        :return: JSON formated MS Teams connector message to send
+        :rtype: str
+        """
+        card = {
+            "themeColor": self.theme_color,
+            "summary": self.message,
+            "sections": [{
+                "activityTitle": self.message,
+                "activitySubtitle": self.subtitle,
+                "activityImage": self.icon_url,
+                "facts": self.facts,
+                "potentialAction": [{
+                    "@context": "http://schema.org",
+                    "@type": "OpenUri",
+                    "name": self.action_button_name,
+                    "targets": [{
+                        "os": "default",
+                        "uri": self.action_button_url
+                    }]
+                }]}
+            ]
+        }
+        return json.dumps(card)
+
+    def execute(self) -> None:

Review Comment:
   > I think there is no need for the teams_ prefix in the file names
   
   Just like slack, we might be adding support for integrating Teams API asides webhook in the future. To maintain naming conventions and avoid filenaming issues in the future, I guess it might be best for this module to be left as teams_webhook.py and the upcoming teams API can then be named teams.py, just like slack providers naming conventions.



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


[GitHub] [airflow] eladkal commented on a diff in pull request #25769: Add support for MS Teams webhook operator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #25769:
URL: https://github.com/apache/airflow/pull/25769#discussion_r949055101


##########
airflow/providers/microsoft/teams/hooks/teams_webhook.py:
##########
@@ -0,0 +1,132 @@
+#
+# 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 typing import Optional
+
+from airflow.exceptions import AirflowException
+from airflow.providers.http.hooks.http import HttpHook
+
+
+class TeamsWebhookHook(HttpHook):
+    """
+    This hook allows you to post messages to MS Teams using the incoming webhooks.
+    Takes both MS Teams webhook token directly and connection that has MS Teams webhook token.
+    If both supplied, the webhook token will be appended to the host in the connection.
+
+    :param http_conn_id: connection that has MS Teams webhook URL
+    :param webhook_token: MS Teams webhook token
+    :param message: The message you want to send on MS Teams
+    :param facts: The facts to send on MS Teams. Should be a list of
+        dictionaries of two keys representing more details to the message.
+        E.g {"name": "Status", "value": "Not started"}
+    :param subtitle: The subtitle of the message to send
+    :param action_button_name: The name of the action button
+    :param action_button_url: The URL for the action button clicked
+    :param theme_color: Hex code of the card theme, without the #
+    :param icon_url: The icon activityImage URL string to be added to message card.
+    :param proxy: Proxy to use to make the MS Teams webhook call
+    """
+
+    def __init__(
+        self,
+        http_conn_id=None,
+        webhook_token=None,
+        message="",
+        subtitle="",
+        theme_color="",
+        facts=None,
+        action_button_name="",
+        action_button_url="",
+        icon_url=None,
+        proxy=None,
+        *args,
+        **kwargs,
+    ):
+        super().__init__(http_conn_id=http_conn_id, *args, **kwargs)
+        self.webhook_token = self._get_token(webhook_token, http_conn_id)
+        self.message = message
+        self.subtitle = subtitle
+        self.theme_color = theme_color
+        self.facts = facts
+        self.action_button_name = action_button_name
+        self.action_button_url = action_button_url
+        self.icon_url = icon_url
+        self.proxy = proxy
+
+    def _get_token(self, token: str, http_conn_id: Optional[str]) -> str:
+        """
+        Given either a manually set token or a conn_id, return the webhook_token to use.
+
+        :param token: The manually provided token
+        :param http_conn_id: The conn_id provided
+        :return: webhook_token to use
+        :rtype: str
+        """
+        if token:
+            return token
+        elif http_conn_id:
+            conn = self.get_connection(http_conn_id)
+            extra = conn.extra_dejson
+            web_token = extra.get('webhook_token', '')
+            return web_token
+        else:
+            raise AirflowException('Cannot get token: No valid Teams webhook token nor conn_id supplied')
+
+    def _build_teams_message(self) -> str:
+        """
+        Construct Teams connector message. All the relevant parameters are combined to a valid
+        Teams json message.
+
+        :return: JSON formated MS Teams connector message to send
+        :rtype: str
+        """
+        card = {
+            "themeColor": self.theme_color,
+            "summary": self.message,
+            "sections": [{
+                "activityTitle": self.message,
+                "activitySubtitle": self.subtitle,
+                "activityImage": self.icon_url,
+                "facts": self.facts,
+                "potentialAction": [{
+                    "@context": "http://schema.org",
+                    "@type": "OpenUri",
+                    "name": self.action_button_name,
+                    "targets": [{
+                        "os": "default",
+                        "uri": self.action_button_url
+                    }]
+                }]}
+            ]
+        }
+        return json.dumps(card)
+
+    def execute(self) -> None:

Review Comment:
   ```suggestion
       def send(self) -> None:
   ```
   
   execute is fine but it may cause confusion with `execute` on Operator classes.
   Lets avoid this potential confusion



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