Skip to content

Commit

Permalink
Remove deprecated parts from Slack provider (#33557)
Browse files Browse the repository at this point in the history
  • Loading branch information
Taragolis committed Aug 24, 2023
1 parent dc47c46 commit ed6a4fd
Show file tree
Hide file tree
Showing 13 changed files with 155 additions and 680 deletions.
38 changes: 38 additions & 0 deletions airflow/providers/slack/CHANGELOG.rst
Expand Up @@ -27,6 +27,44 @@
Changelog
---------

8.0.0
.....

Breaking changes
~~~~~~~~~~~~~~~~

.. warning::
``SlackHook`` and ``SlackWebhookHook`` constructor expected keyword-only arguments.

Removed deprecated parameter ``token`` from the ``SlackHook`` and dependent operators.
Required create ``Slack API Connection`` and provide connection id to ``slack_conn_id`` operators / hook,
and the behavior should stay the same.

Parsing Slack Incoming Webhook Token from the Connection ``hostname`` is removed, ``password`` should be filled.

Removed deprecated parameter ``webhook_token`` from the ``SlackWebhookHook`` and dependent operators
Required create ``Slack Incoming Webhook Connection`` and provide connection id to ``slack_webhook_conn_id``
operators / hook, and the behavior should stay the same.

Removed deprecated method ``execute`` from the ``SlackWebhookHook``. Use ``send``, ``send_text`` or ``send_dict`` instead.

Removed deprecated parameters ``attachments``, ``blocks``, ``channel``, ``username``, ``username``,
``icon_emoji`` from the ``SlackWebhookHook``. Provide them directly to ``SlackWebhookHook.send`` method,
and the behavior should stay the same.

Removed deprecated parameter ``message`` from the ``SlackWebhookHook``.
Provide ``text`` directly to ``SlackWebhookHook.send`` method, and the behavior should stay the same.

Removed deprecated parameter ``link_names`` from the ``SlackWebhookHook`` and dependent operators.
This parameter has no affect in the past, you should not provide it.
If you want to mention user see: `Slack Documentation <https://api.slack.com/reference/surfaces/formatting#mentioning-users>`__.

Removed deprecated parameters ``endpoint``, ``method``, ``data``, ``headers``, ``response_check``,
``response_filter``, ``extra_options``, ``log_response``, ``auth_type``, ``tcp_keep_alive``,
``tcp_keep_alive_idle``, ``tcp_keep_alive_idle``, ``tcp_keep_alive_count``, ``tcp_keep_alive_interval``
from the ``SlackWebhookOperator``. Those parameters has no affect in the past, you should not provide it.


7.3.2
.....

Expand Down
93 changes: 20 additions & 73 deletions airflow/providers/slack/hooks/slack.py
Expand Up @@ -26,11 +26,10 @@
from slack_sdk import WebClient
from slack_sdk.errors import SlackApiError

from airflow.exceptions import AirflowException, AirflowNotFoundException, AirflowProviderDeprecationWarning
from airflow.exceptions import AirflowNotFoundException
from airflow.hooks.base import BaseHook
from airflow.providers.slack.utils import ConnectionExtraConfig
from airflow.utils.helpers import exactly_one
from airflow.utils.log.secrets_masker import mask_secret

if TYPE_CHECKING:
from slack_sdk.http_retry import RetryHandler
Expand Down Expand Up @@ -103,7 +102,6 @@ class SlackHook(BaseHook):
If not set than default WebClient BASE_URL will use (``https://www.slack.com/api/``).
:param proxy: Proxy to make the Slack API call.
:param retry_handlers: List of handlers to customize retry logic in ``slack_sdk.WebClient``.
:param token: (deprecated) Slack API Token.
"""

conn_name_attr = "slack_conn_id"
Expand All @@ -113,42 +111,31 @@ class SlackHook(BaseHook):

def __init__(
self,
token: str | None = None,
slack_conn_id: str | None = None,
*,
slack_conn_id: str,
base_url: str | None = None,
timeout: int | None = None,
proxy: str | None = None,
retry_handlers: list[RetryHandler] | None = None,
**extra_client_args: Any,
) -> None:
if not token and not slack_conn_id:
raise AirflowException("Either `slack_conn_id` or `token` should be provided.")
if token:
mask_secret(token)
warnings.warn(
"Provide token as hook argument deprecated by security reason and will be removed "
"in a future releases. Please specify token in `Slack API` connection.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
if not slack_conn_id:
warnings.warn(
"You have not set parameter `slack_conn_id`. Currently `Slack API` connection id optional "
"but in a future release it will mandatory.",
FutureWarning,
stacklevel=2,
)

super().__init__()
self._token = token
self.slack_conn_id = slack_conn_id
self.base_url = base_url
self.timeout = timeout
self.proxy = proxy
self.retry_handlers = retry_handlers
if "token" in extra_client_args:
warnings.warn(
f"Provide `token` as part of {type(self).__name__!r} parameters is disallowed, "
f"please use Airflow Connection.",
UserWarning,
stacklevel=2,
)
extra_client_args.pop("token")
if "logger" not in extra_client_args:
extra_client_args["logger"] = self.log
self.extra_client_args = extra_client_args
if self.extra_client_args.pop("use_session", None) is not None:
warnings.warn("`use_session` has no affect in slack_sdk.WebClient.", UserWarning, stacklevel=2)

@cached_property
def client(self) -> WebClient:
Expand All @@ -161,24 +148,15 @@ def get_conn(self) -> WebClient:

def _get_conn_params(self) -> dict[str, Any]:
"""Fetch connection params as a dict and merge it with hook parameters."""
conn = self.get_connection(self.slack_conn_id) if self.slack_conn_id else None
conn_params: dict[str, Any] = {"retry_handlers": self.retry_handlers}

if self._token:
conn_params["token"] = self._token
elif conn:
if not conn.password:
raise AirflowNotFoundException(
f"Connection ID {self.slack_conn_id!r} does not contain password (Slack API Token)."
)
conn_params["token"] = conn.password

conn = self.get_connection(self.slack_conn_id)
if not conn.password:
raise AirflowNotFoundException(
f"Connection ID {self.slack_conn_id!r} does not contain password (Slack API Token)."
)
conn_params: dict[str, Any] = {"token": conn.password, "retry_handlers": self.retry_handlers}
extra_config = ConnectionExtraConfig(
conn_type=self.conn_type,
conn_id=conn.conn_id if conn else None,
extra=conn.extra_dejson if conn else {},
conn_type=self.conn_type, conn_id=conn.conn_id, extra=conn.extra_dejson
)

# Merge Hook parameters with Connection config
conn_params.update(
{
Expand All @@ -187,41 +165,10 @@ def _get_conn_params(self) -> dict[str, Any]:
"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}

@cached_property
def token(self) -> str:
warnings.warn(
"`SlackHook.token` property deprecated and will be removed in a future releases.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
return self._get_conn_params()["token"]

def __get_token(self, token: Any, slack_conn_id: Any) -> str:
warnings.warn(
"`SlackHook.__get_token` method deprecated and will be removed in a future releases.",
AirflowProviderDeprecationWarning,
stacklevel=2,
)
if token is not None:
return token

if slack_conn_id is not None:
conn = self.get_connection(slack_conn_id)

if not getattr(conn, "password", None):
raise AirflowException("Missing token(password) in Slack connection")
return conn.password

raise AirflowException("Cannot get token: No valid Slack token nor slack_conn_id supplied.")

def call(self, api_method: str, **kwargs) -> SlackResponse:
"""
Calls Slack WebClient `WebClient.api_call` with given arguments.
Expand Down

0 comments on commit ed6a4fd

Please sign in to comment.