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/09/17 13:23:11 UTC

[GitHub] [airflow] Taragolis opened a new pull request, #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

Taragolis opened a new pull request, #26452:
URL: https://github.com/apache/airflow/pull/26452

   Fundamental refactor of SlackWebhookHook with breaking changes.
   I tried to minimise breaking changes, however main (and hope the only once) braking change SlackWebhookHook not anymore inherit from `airflow.providers.http.hooks.http.HttpHook` anymore and use slack_sdk.WebhookClient instead
   
   Additional changes:
   1. Warn user that it is not safe to specify webhook token (url) directly in the Hook and suggest to switch to Airflow Connections.
   2. Deprecate specify webhook message attribute in `__init__` method and mainly use hook attributes only for configure slack_sdk.WebhookClient. It is still possible to set as Hook arguments.
   3. Inform users (always) that it is not possible to change channel, username and icon by use Slack Incoming Webhook. Users might previously use for two reasons:
      a. Hook and Operators not cover this part
      b. Hook initially created for use with Legacy Incoming Webhooks based on Slack Integration
   4. Change SlackWebhookHook arguments in Operators 
   5. Add documentation for Slack Incoming Webhook connection type
   
   ![image](https://user-images.githubusercontent.com/3998685/190858233-61a9a043-bd73-4f6c-9819-25610ae81f9d.png)
   


-- 
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 a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)

Review Comment:
   good idea.



-- 
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 #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   @Taragolis - are you working on making some of the changes discussed ? I am gearing up to prepare a new provider's wave, so it would be great to merge this one before.


-- 
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 a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)
+            url = (base_url.rstrip("/") + "/" + webhook_token.lstrip("/")).rstrip("/")
+
+        conn_params["url"] = url
+        # Merge Hook parameters with Connection config
+        conn_params.update(
+            {
+                "timeout": self.timeout or extra_config.getint("timeout", default=None),
+                "proxy": self.proxy or extra_config.get("proxy", default=None),
+            }
+        )
+        # Add additional client args
+        conn_params.update(self.extra_client_args)
+        if "logger" not in conn_params:
+            conn_params["logger"] = self.log
+
+        return {k: v for k, v in conn_params.items() if v is not None}
+
+    def _resolve_argument(self, name: str, value):
         """
-        Construct the Slack message. All relevant parameters are combined here to a valid
-        Slack json message.
+        Resolve message parameters.
 
-        :return: Slack message to send
-        :rtype: str
+        .. note::
+            This method exist for compatibility and merge instance class attributes with
+            method attributes and not be required when assign class attributes to message
+            would completely remove.
         """
-        cmd = {}
-
-        if self.channel:
-            cmd['channel'] = self.channel
-        if self.username:
-            cmd['username'] = self.username
-        if self.icon_emoji:
-            cmd['icon_emoji'] = self.icon_emoji
-        if self.icon_url:
-            cmd['icon_url'] = self.icon_url
-        if self.link_names:
-            cmd['link_names'] = 1
-        if self.attachments:
-            cmd['attachments'] = self.attachments
-        if self.blocks:
-            cmd['blocks'] = self.blocks
-
-        cmd['text'] = self.message
-        return json.dumps(cmd)
+        if value is None and name in (
+            "text",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            return getattr(self, name, None)
+
+        return value
+
+    @check_webhook_response
+    def send_dict(self, body: dict[str, Any] | str, *, headers: dict[str, str] | None = None):
+        """
+        Performs a Slack Incoming Webhook request with given JSON data block.
+
+        :param body: JSON data structure, expected dict or JSON-string.
+        :param headers: Request headers for this request.
+        """
+        if isinstance(body, str):
+            try:
+                body = json.loads(body)
+            except json.JSONDecodeError as err:
+                raise AirflowException(
+                    f"Body expected valid JSON string, got {body!r}. Original error:\n * {err}"
+                ) from None
+
+        if not isinstance(body, dict):
+            raise TypeError(f"Body expected dictionary, got {type(body).__name__}.")
+
+        if "link_names" in body:
+            warnings.warn(
+                "`link_names` has no affect, if you want to mention user see: "
+                "https://api.slack.com/reference/surfaces/formatting#mentioning-users",
+                UserWarning,
+                stacklevel=2,
+            )

Review Comment:
   One thing with such moves is to make sure this check is done only in execute() and Hook is only initialized in execute if the parameter to verify is templateble. I think this is not a problem here, so yeah - moving to constructor makes sense.



-- 
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] Taragolis commented on pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   > @Taragolis - are you working on making some of the changes discussed ? I am gearing up to prepare a new provider's wave, so it would be great to merge this one before.
   
   Yep. Plan to finish changes by today


-- 
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 #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   BTW. Thanks from trying to unentangle the mess :D


-- 
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] Taragolis commented on pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   Finally all checks passed. Today it takes a bit longer rather than usual.


-- 
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 merged pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

Posted by GitBox <gi...@apache.org>.
potiuk merged PR #26452:
URL: https://github.com/apache/airflow/pull/26452


-- 
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 #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   Slack seems to be a beast - can you plese rebase :) ? 


-- 
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 a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)
+            url = (base_url.rstrip("/") + "/" + webhook_token.lstrip("/")).rstrip("/")
+
+        conn_params["url"] = url
+        # Merge Hook parameters with Connection config
+        conn_params.update(
+            {
+                "timeout": self.timeout or extra_config.getint("timeout", default=None),
+                "proxy": self.proxy or extra_config.get("proxy", default=None),
+            }
+        )
+        # Add additional client args
+        conn_params.update(self.extra_client_args)
+        if "logger" not in conn_params:
+            conn_params["logger"] = self.log
+
+        return {k: v for k, v in conn_params.items() if v is not None}
+
+    def _resolve_argument(self, name: str, value):
         """
-        Construct the Slack message. All relevant parameters are combined here to a valid
-        Slack json message.
+        Resolve message parameters.
 
-        :return: Slack message to send
-        :rtype: str
+        .. note::
+            This method exist for compatibility and merge instance class attributes with
+            method attributes and not be required when assign class attributes to message
+            would completely remove.
         """
-        cmd = {}
-
-        if self.channel:
-            cmd['channel'] = self.channel
-        if self.username:
-            cmd['username'] = self.username
-        if self.icon_emoji:
-            cmd['icon_emoji'] = self.icon_emoji
-        if self.icon_url:
-            cmd['icon_url'] = self.icon_url
-        if self.link_names:
-            cmd['link_names'] = 1
-        if self.attachments:
-            cmd['attachments'] = self.attachments
-        if self.blocks:
-            cmd['blocks'] = self.blocks
-
-        cmd['text'] = self.message
-        return json.dumps(cmd)
+        if value is None and name in (
+            "text",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            return getattr(self, name, None)
+
+        return value
+
+    @check_webhook_response
+    def send_dict(self, body: dict[str, Any] | str, *, headers: dict[str, str] | None = None):
+        """
+        Performs a Slack Incoming Webhook request with given JSON data block.
+
+        :param body: JSON data structure, expected dict or JSON-string.
+        :param headers: Request headers for this request.
+        """
+        if isinstance(body, str):
+            try:
+                body = json.loads(body)
+            except json.JSONDecodeError as err:
+                raise AirflowException(
+                    f"Body expected valid JSON string, got {body!r}. Original error:\n * {err}"
+                ) from None
+
+        if not isinstance(body, dict):
+            raise TypeError(f"Body expected dictionary, got {type(body).__name__}.")
+
+        if "link_names" in body:
+            warnings.warn(
+                "`link_names` has no affect, if you want to mention user see: "
+                "https://api.slack.com/reference/surfaces/formatting#mentioning-users",
+                UserWarning,
+                stacklevel=2,
+            )
+        if any(legacy_attr in body for legacy_attr in ("channel", "username", "icon_emoji", "icon_url")):
+            warnings.warn(
+                "You cannot override the default channel (chosen by the user who installed your app), "
+                "username, or icon when you're using Incoming Webhooks to post messages. "
+                "Instead, these values will always inherit from the associated Slack app configuration. "
+                "See: https://api.slack.com/messaging/webhooks#advanced_message_formatting. "
+                "It is possible to change this values only in Legacy Slack Integration Incoming Webhook: "
+                "https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations",
+                UserWarning,
+                stacklevel=2,
+            )

Review Comment:
   Is there are good reason why it would not be possible, or just " I had no time to change it?". If that's the latter, then this is no problem to have warnings. Warnings are there to be annoying enough to gently push the user to get rid of the legacy code and make investment in doing so. The only reason for allowing to silence warnings easily is when there migtht be really good reasons and use cases that cannot be handled with the "new way" of doing things. 



-- 
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] Taragolis commented on a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)

Review Comment:
   `webhook_token` in Connection extra deprecated since slack provider `1.0.0`, so might be also a good time to drop support this.
   
   https://github.com/apache/airflow/blob/db166ba75c447a08b94e7be1ab09042fd6361581/airflow/providers/slack/hooks/slack_webhook.py#L115-L120



##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)

Review Comment:
   This condition for support Slack Webhook URL in conn.host without password.
   
   https://github.com/apache/airflow/blob/02d22f6ce2dbb4a1c5c5eb01dfa3070327e377bb/tests/providers/slack/hooks/test_slack_webhook.py#L69-L73
   
   Might be also deprecate this option since it allow expose this value in Connection UI
   
   ![image](https://user-images.githubusercontent.com/3998685/191041634-7480dc2f-6540-4b57-844f-760c44e848c7.png)
   
   



##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)
+            url = (base_url.rstrip("/") + "/" + webhook_token.lstrip("/")).rstrip("/")
+
+        conn_params["url"] = url
+        # Merge Hook parameters with Connection config
+        conn_params.update(
+            {
+                "timeout": self.timeout or extra_config.getint("timeout", default=None),
+                "proxy": self.proxy or extra_config.get("proxy", default=None),
+            }
+        )
+        # Add additional client args
+        conn_params.update(self.extra_client_args)
+        if "logger" not in conn_params:
+            conn_params["logger"] = self.log
+
+        return {k: v for k, v in conn_params.items() if v is not None}
+
+    def _resolve_argument(self, name: str, value):
         """
-        Construct the Slack message. All relevant parameters are combined here to a valid
-        Slack json message.
+        Resolve message parameters.
 
-        :return: Slack message to send
-        :rtype: str
+        .. note::
+            This method exist for compatibility and merge instance class attributes with
+            method attributes and not be required when assign class attributes to message
+            would completely remove.
         """
-        cmd = {}
-
-        if self.channel:
-            cmd['channel'] = self.channel
-        if self.username:
-            cmd['username'] = self.username
-        if self.icon_emoji:
-            cmd['icon_emoji'] = self.icon_emoji
-        if self.icon_url:
-            cmd['icon_url'] = self.icon_url
-        if self.link_names:
-            cmd['link_names'] = 1
-        if self.attachments:
-            cmd['attachments'] = self.attachments
-        if self.blocks:
-            cmd['blocks'] = self.blocks
-
-        cmd['text'] = self.message
-        return json.dumps(cmd)
+        if value is None and name in (
+            "text",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            return getattr(self, name, None)
+
+        return value
+
+    @check_webhook_response
+    def send_dict(self, body: dict[str, Any] | str, *, headers: dict[str, str] | None = None):
+        """
+        Performs a Slack Incoming Webhook request with given JSON data block.
+
+        :param body: JSON data structure, expected dict or JSON-string.
+        :param headers: Request headers for this request.
+        """
+        if isinstance(body, str):
+            try:
+                body = json.loads(body)
+            except json.JSONDecodeError as err:
+                raise AirflowException(
+                    f"Body expected valid JSON string, got {body!r}. Original error:\n * {err}"
+                ) from None
+
+        if not isinstance(body, dict):
+            raise TypeError(f"Body expected dictionary, got {type(body).__name__}.")
+
+        if "link_names" in body:
+            warnings.warn(
+                "`link_names` has no affect, if you want to mention user see: "
+                "https://api.slack.com/reference/surfaces/formatting#mentioning-users",
+                UserWarning,
+                stacklevel=2,
+            )

Review Comment:
   I also think about move this check into hook initialise



##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)
+            url = (base_url.rstrip("/") + "/" + webhook_token.lstrip("/")).rstrip("/")
+
+        conn_params["url"] = url
+        # Merge Hook parameters with Connection config
+        conn_params.update(
+            {
+                "timeout": self.timeout or extra_config.getint("timeout", default=None),
+                "proxy": self.proxy or extra_config.get("proxy", default=None),
+            }
+        )
+        # Add additional client args
+        conn_params.update(self.extra_client_args)
+        if "logger" not in conn_params:
+            conn_params["logger"] = self.log
+
+        return {k: v for k, v in conn_params.items() if v is not None}
+
+    def _resolve_argument(self, name: str, value):
         """
-        Construct the Slack message. All relevant parameters are combined here to a valid
-        Slack json message.
+        Resolve message parameters.
 
-        :return: Slack message to send
-        :rtype: str
+        .. note::
+            This method exist for compatibility and merge instance class attributes with
+            method attributes and not be required when assign class attributes to message
+            would completely remove.
         """
-        cmd = {}
-
-        if self.channel:
-            cmd['channel'] = self.channel
-        if self.username:
-            cmd['username'] = self.username
-        if self.icon_emoji:
-            cmd['icon_emoji'] = self.icon_emoji
-        if self.icon_url:
-            cmd['icon_url'] = self.icon_url
-        if self.link_names:
-            cmd['link_names'] = 1
-        if self.attachments:
-            cmd['attachments'] = self.attachments
-        if self.blocks:
-            cmd['blocks'] = self.blocks
-
-        cmd['text'] = self.message
-        return json.dumps(cmd)
+        if value is None and name in (
+            "text",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            return getattr(self, name, None)
+
+        return value
+
+    @check_webhook_response
+    def send_dict(self, body: dict[str, Any] | str, *, headers: dict[str, str] | None = None):
+        """
+        Performs a Slack Incoming Webhook request with given JSON data block.
+
+        :param body: JSON data structure, expected dict or JSON-string.
+        :param headers: Request headers for this request.
+        """
+        if isinstance(body, str):
+            try:
+                body = json.loads(body)
+            except json.JSONDecodeError as err:
+                raise AirflowException(
+                    f"Body expected valid JSON string, got {body!r}. Original error:\n * {err}"
+                ) from None
+
+        if not isinstance(body, dict):
+            raise TypeError(f"Body expected dictionary, got {type(body).__name__}.")
+
+        if "link_names" in body:
+            warnings.warn(
+                "`link_names` has no affect, if you want to mention user see: "
+                "https://api.slack.com/reference/surfaces/formatting#mentioning-users",
+                UserWarning,
+                stacklevel=2,
+            )
+        if any(legacy_attr in body for legacy_attr in ("channel", "username", "icon_emoji", "icon_url")):
+            warnings.warn(
+                "You cannot override the default channel (chosen by the user who installed your app), "
+                "username, or icon when you're using Incoming Webhooks to post messages. "
+                "Instead, these values will always inherit from the associated Slack app configuration. "
+                "See: https://api.slack.com/messaging/webhooks#advanced_message_formatting. "
+                "It is possible to change this values only in Legacy Slack Integration Incoming Webhook: "
+                "https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations",
+                UserWarning,
+                stacklevel=2,
+            )

Review Comment:
   Another option create separate methods `send_legacy` and `send_text_legacy` which allow send to Legacy Incoming Webhook without any warnings, might be helpful if there is no possible for user change Legacy Incoming Webhook.



-- 
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] Taragolis commented on a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)
 
-                return web_token
+        webhook_token = webhook_token or ""
+        if not webhook_token and not conn.host:
+            raise AirflowException("Cannot get token: No valid Slack token nor valid Connection ID supplied.")
+        elif webhook_token and "://" in webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token URL from webhook token.")
+            url = webhook_token
         else:
-            raise AirflowException('Cannot get token: No valid Slack webhook token nor conn_id supplied')
+            self.log.debug("Constructing Slack Webhook Token URL.")
+            if conn.host and "://" in conn.host:
+                base_url = conn.host
+            else:
+                schema = conn.schema if conn.schema else default_schema
+                host = conn.host if conn.host else default_host
+                base_url = f"{schema}://{host}"
 
-    def _build_slack_message(self) -> str:
+            base_url = base_url.rstrip("/")
+            if not webhook_token:
+                parsed_token = (urlparse(base_url).path or "").strip("/")
+                if base_url == DEFAULT_SLACK_WEBHOOK_ENDPOINT or not parsed_token:
+                    # Raise an error in case of password not specified and
+                    # 1. Result of constructing base_url equal https://hooks.slack.com/services
+                    # 2. Empty url path, e.g. if base_url = https://hooks.slack.com
+                    raise AirflowException(
+                        "Cannot get token: No valid Slack token nor valid Connection ID supplied."
+                    )
+                mask_secret(parsed_token)
+            url = (base_url.rstrip("/") + "/" + webhook_token.lstrip("/")).rstrip("/")
+
+        conn_params["url"] = url
+        # Merge Hook parameters with Connection config
+        conn_params.update(
+            {
+                "timeout": self.timeout or extra_config.getint("timeout", default=None),
+                "proxy": self.proxy or extra_config.get("proxy", default=None),
+            }
+        )
+        # Add additional client args
+        conn_params.update(self.extra_client_args)
+        if "logger" not in conn_params:
+            conn_params["logger"] = self.log
+
+        return {k: v for k, v in conn_params.items() if v is not None}
+
+    def _resolve_argument(self, name: str, value):
         """
-        Construct the Slack message. All relevant parameters are combined here to a valid
-        Slack json message.
+        Resolve message parameters.
 
-        :return: Slack message to send
-        :rtype: str
+        .. note::
+            This method exist for compatibility and merge instance class attributes with
+            method attributes and not be required when assign class attributes to message
+            would completely remove.
         """
-        cmd = {}
-
-        if self.channel:
-            cmd['channel'] = self.channel
-        if self.username:
-            cmd['username'] = self.username
-        if self.icon_emoji:
-            cmd['icon_emoji'] = self.icon_emoji
-        if self.icon_url:
-            cmd['icon_url'] = self.icon_url
-        if self.link_names:
-            cmd['link_names'] = 1
-        if self.attachments:
-            cmd['attachments'] = self.attachments
-        if self.blocks:
-            cmd['blocks'] = self.blocks
-
-        cmd['text'] = self.message
-        return json.dumps(cmd)
+        if value is None and name in (
+            "text",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            return getattr(self, name, None)
+
+        return value
+
+    @check_webhook_response
+    def send_dict(self, body: dict[str, Any] | str, *, headers: dict[str, str] | None = None):
+        """
+        Performs a Slack Incoming Webhook request with given JSON data block.
+
+        :param body: JSON data structure, expected dict or JSON-string.
+        :param headers: Request headers for this request.
+        """
+        if isinstance(body, str):
+            try:
+                body = json.loads(body)
+            except json.JSONDecodeError as err:
+                raise AirflowException(
+                    f"Body expected valid JSON string, got {body!r}. Original error:\n * {err}"
+                ) from None
+
+        if not isinstance(body, dict):
+            raise TypeError(f"Body expected dictionary, got {type(body).__name__}.")
+
+        if "link_names" in body:
+            warnings.warn(
+                "`link_names` has no affect, if you want to mention user see: "
+                "https://api.slack.com/reference/surfaces/formatting#mentioning-users",
+                UserWarning,
+                stacklevel=2,
+            )
+        if any(legacy_attr in body for legacy_attr in ("channel", "username", "icon_emoji", "icon_url")):
+            warnings.warn(
+                "You cannot override the default channel (chosen by the user who installed your app), "
+                "username, or icon when you're using Incoming Webhooks to post messages. "
+                "Instead, these values will always inherit from the associated Slack app configuration. "
+                "See: https://api.slack.com/messaging/webhooks#advanced_message_formatting. "
+                "It is possible to change this values only in Legacy Slack Integration Incoming Webhook: "
+                "https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations",
+                UserWarning,
+                stacklevel=2,
+            )

Review Comment:
   Only one case when user do not have access to create new applications in slack workspace. 
   One day `Slack Webhook based on Slack integrations` will be removed anyway so in user perspective better started to migrated from legacy service to 2 successors: Slack API and Slack Incoming Webhook based on API/App
   
   This might be the later feature if someone requested it and better warn user right now always.



-- 
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 a diff in pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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


##########
airflow/providers/slack/hooks/slack_webhook.py:
##########
@@ -19,144 +19,437 @@
 
 import json
 import warnings
+from functools import wraps
+from typing import TYPE_CHECKING, Any, Callable
+from urllib.parse import urlparse
 
+from slack_sdk import WebhookClient
+
+from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException
-from airflow.providers.http.hooks.http import HttpHook
+from airflow.hooks.base import BaseHook
+from airflow.models import Connection
+from airflow.providers.slack.utils import ConnectionExtraConfig
+from airflow.utils.log.secrets_masker import mask_secret
+
+if TYPE_CHECKING:
+    from slack_sdk.http_retry import RetryHandler
+
+DEFAULT_SLACK_WEBHOOK_ENDPOINT = "https://hooks.slack.com/services"
+LEGACY_INTEGRATION_PARAMS = ("channel", "username", "icon_emoji", "icon_url")
+
 
+def check_webhook_response(func: Callable) -> Callable:
+    """Function decorator that check WebhookResponse and raise an error if status code != 200."""
 
-class SlackWebhookHook(HttpHook):
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        resp = func(*args, **kwargs)
+        if resp.status_code != 200:
+            raise AirflowException(
+                f"Response body: {resp.body!r}, Status Code: {resp.status_code}. "
+                "See: https://api.slack.com/messaging/webhooks#handling_errors"
+            )
+        return resp
+
+    return wrapper
+
+
+class SlackWebhookHook(BaseHook):
     """
-    This hook allows you to post messages to Slack using incoming webhooks.
-    Takes both Slack webhook token directly and connection that has Slack webhook token.
-    If both supplied, http_conn_id will be used as base_url,
-    and webhook_token will be taken as endpoint, the relative path of the url.
+    This class provide a thin wrapper around the ``slack_sdk.WebhookClient``.
+    This hook allows you to post messages to Slack by using Incoming Webhooks.
+
+    .. seealso::
+        - :ref:`Slack Incoming Webhook connection <howto/connection:slack-incoming-webhook>`
+        - https://api.slack.com/messaging/webhooks
+        - https://slack.dev/python-slack-sdk/webhook/index.html
+
+    .. note::
+        You cannot override the default channel (chosen by the user who installed your app),
+        username, or icon when you're using Incoming Webhooks to post messages.
+        Instead, these values will always inherit from the associated Slack App configuration
+        (`link <https://api.slack.com/messaging/webhooks#advanced_message_formatting>`_).
+        It is possible to change this values only in `Legacy Slack Integration Incoming Webhook
+        <https://api.slack.com/legacy/custom-integrations/messaging/webhooks#legacy-customizations>`_.
 
     .. warning::
-        This hook intend to use `Slack Webhook` connection
+        This hook intend to use `Slack Incoming Webhook` connection
         and might not work correctly with `Slack API` connection.
 
-    Each Slack webhook token can be pre-configured to use a specific channel, username and
-    icon. You can override these defaults in this hook.
-
-    :param http_conn_id: connection that has Slack webhook token in the password field
-    :param webhook_token: Slack webhook token
-    :param message: The message you want to send on Slack
-    :param attachments: The attachments to send on Slack. Should be a list of
-        dictionaries representing Slack attachments.
-    :param blocks: The blocks to send on Slack. Should be a list of
-        dictionaries representing Slack blocks.
-    :param channel: The channel the message should be posted to
-    :param username: The username to post to slack with
-    :param icon_emoji: The emoji to use as icon for the user posting to Slack
-    :param icon_url: The icon image URL string to use in place of the default icon.
-    :param link_names: Whether or not to find and link channel and usernames in your
-        message
-    :param proxy: Proxy to use to make the Slack webhook call
+    Examples:
+     .. code-block:: python
+
+        # Create hook
+        hook = SlackWebhookHook(slack_webhook_conn_id="slack_default")
+
+        # Post message in Slack channel by JSON formatted message
+        # See: https://api.slack.com/messaging/webhooks#posting_with_webhooks
+        hook.send_dict({"text": "Hello world!"})
+
+        # Post simple message in Slack channel
+        hook.send_text("Hello world!")
+
+        # Use ``slack_sdk.WebhookClient``
+        hook.client.send(text="Hello world!")
+
+    :param slack_webhook_conn_id: Slack Incoming Webhook connection id
+        that has Incoming Webhook token in the password field.
+    :param timeout: The maximum number of seconds the client will wait to connect
+        and receive a response from Slack. If not set than default WebhookClient value will use.
+    :param proxy: Proxy to make the Slack Incoming Webhook call.
+    :param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebhookClient``.
+    :param webhook_token: (deprecated) Slack Incoming Webhook token.
+        Use instead Slack Incoming Webhook connection password field.
     """
 
-    conn_name_attr = 'http_conn_id'
+    conn_name_attr = 'slack_webhook_conn_id'
     default_conn_name = 'slack_default'
     conn_type = 'slackwebhook'
-    hook_name = 'Slack Webhook'
+    hook_name = 'Slack Incoming Webhook'
 
     def __init__(
         self,
-        http_conn_id=None,
-        webhook_token=None,
-        message="",
-        attachments=None,
-        blocks=None,
-        channel=None,
-        username=None,
-        icon_emoji=None,
-        icon_url=None,
-        link_names=False,
-        proxy=None,
-        *args,
+        slack_webhook_conn_id: str | None = None,
+        webhook_token: str | None = None,
+        timeout: int | None = None,
+        proxy: str | None = None,
+        retry_handlers: list[RetryHandler] | None = None,
         **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.attachments = attachments
-        self.blocks = blocks
-        self.channel = channel
-        self.username = username
-        self.icon_emoji = icon_emoji
-        self.icon_url = icon_url
-        self.link_names = link_names
+        super().__init__()
+
+        http_conn_id = kwargs.pop("http_conn_id", None)
+        if http_conn_id:
+            warnings.warn(
+                'Parameter `http_conn_id` is deprecated. Please use `slack_webhook_conn_id` instead.',
+                DeprecationWarning,
+                stacklevel=2,
+            )
+            if slack_webhook_conn_id:
+                raise AirflowException("You cannot provide both `slack_webhook_conn_id` and `http_conn_id`.")
+            slack_webhook_conn_id = http_conn_id
+
+        if not slack_webhook_conn_id and not webhook_token:
+            raise AirflowException("Either `slack_webhook_conn_id` or `webhook_token` should be provided.")
+        if webhook_token:
+            mask_secret(webhook_token)
+            warnings.warn(
+                "Provide `webhook_token` as hook argument deprecated by security reason and will be removed "
+                "in a future releases. Please specify it in `Slack Webhook` connection.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
+        if not slack_webhook_conn_id:
+            warnings.warn(
+                "You have not set parameter `slack_webhook_conn_id`. Currently `Slack Incoming Webhook` "
+                "connection id optional but in a future release it will mandatory.",
+                FutureWarning,
+                stacklevel=2,
+            )
+
+        self.slack_webhook_conn_id = slack_webhook_conn_id
+        self.timeout = timeout
         self.proxy = proxy
+        self.retry_handlers = retry_handlers
+        self._webhook_token = webhook_token
 
-    def _get_token(self, token: str, http_conn_id: str | None) -> str:
-        """
-        Given either a manually set token or a conn_id, return the webhook_token to use.
+        # Compatibility with previous version of SlackWebhookHook
+        deprecated_class_attrs = []
+        for deprecated_attr in (
+            "message",
+            "attachments",
+            "blocks",
+            "channel",
+            "username",
+            "icon_emoji",
+            "icon_url",
+            "link_names",
+        ):
+            if deprecated_attr in kwargs:
+                deprecated_class_attrs.append(deprecated_attr)
+                setattr(self, deprecated_attr, kwargs.pop(deprecated_attr))
+                if deprecated_attr == "message":
+                    # Slack WebHook Post Request not expected `message` as field,
+                    # so we also set "text" attribute which will check by SlackWebhookHook._resolve_argument
+                    self.text = getattr(self, deprecated_attr)
 
-        :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)
+        if deprecated_class_attrs:
+            warnings.warn(
+                f"Provide {','.join(repr(a) for a in deprecated_class_attrs)} as hook argument(s) "
+                f"is deprecated and will be removed in a future releases. "
+                f"Please specify attributes in `{self.__class__.__name__}.send` method instead.",
+                DeprecationWarning,
+                stacklevel=2,
+            )
 
-            if getattr(conn, 'password', None):
-                return conn.password
-            else:
-                extra = conn.extra_dejson
-                web_token = extra.get('webhook_token', '')
+        self.extra_client_args = kwargs
+
+    @cached_property
+    def client(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return WebhookClient(**self._get_conn_params())
 
-                if web_token:
+    def get_conn(self) -> WebhookClient:
+        """Get the underlying slack_sdk.webhook.WebhookClient (cached)."""
+        return self.client
+
+    @cached_property
+    def webhook_token(self) -> str:
+        """Return Slack Webhook Token URL."""
+        warnings.warn(
+            "`SlackHook.webhook_token` property deprecated and will be removed in a future releases.",
+            DeprecationWarning,
+            stacklevel=2,
+        )
+        return self._get_conn_params()["url"]
+
+    def _get_conn_params(self) -> dict[str, Any]:
+        """Fetch connection params as a dict and merge it with hook parameters."""
+        default_schema, _, default_host = DEFAULT_SLACK_WEBHOOK_ENDPOINT.partition("://")
+        if self.slack_webhook_conn_id:
+            conn = self.get_connection(self.slack_webhook_conn_id)
+        else:
+            # If slack_webhook_conn_id not specified, then use connection with default schema and host
+            conn = Connection(
+                conn_id=None, conn_type=self.conn_type, host=default_schema, password=default_host
+            )
+        extra_config = ConnectionExtraConfig(
+            conn_type=self.conn_type,
+            conn_id=conn.conn_id,
+            extra=conn.extra_dejson,
+        )
+        conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}
+
+        webhook_token = None
+        if self._webhook_token:
+            self.log.debug("Retrieving Slack Webhook Token from hook attribute.")
+            webhook_token = self._webhook_token
+        elif conn.conn_id:
+            if conn.password:
+                self.log.debug(
+                    "Retrieving Slack Webhook Token from Connection ID %r password.",
+                    self.slack_webhook_conn_id,
+                )
+                webhook_token = conn.password
+            else:
+                webhook_token = extra_config.get("webhook_token", None)
+                if webhook_token:
                     warnings.warn(
-                        "'webhook_token' in 'extra' is deprecated. Please use 'password' field",
+                        f"Found 'webhook_token' in Connection {conn.conn_id!r} Extra, this option is "
+                        "deprecated and will be removed in a future releases. Please use 'password' field.",
                         DeprecationWarning,
                         stacklevel=2,
                     )
+                    mask_secret(webhook_token)

Review Comment:
   Yep. Why not.



-- 
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] Taragolis commented on pull request #26452: Refactor SlackWebhookHook in order to use `slack_sdk` instead of HttpHook methods

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

   Even with this PR it is not possible to get rid of `apache-airflow-providers-http` dependency.
   Mainly because `SlackWebhookOperator` inherit from `SimpleHttpOperator`
   https://github.com/apache/airflow/blob/706a618014a6f94d5ead0476f26f79d9714bf93d/airflow/providers/slack/operators/slack_webhook.py#L28-L30
   
   But actually it not use any feature of `SimpleHttpOperator` just because it overwrite execute method where all internal stuff of `SimpleHttpOperator` happen.
   
   Changes in `SlackWebhookOperator` could be done after this PR. In fact this PR contain a lot of changes so it would be difficult to track changes if it done in single PR.
   https://github.com/apache/airflow/blob/706a618014a6f94d5ead0476f26f79d9714bf93d/airflow/providers/slack/operators/slack_webhook.py#L94-L110


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