From 285f9ab6b64a66f985f41796f5cff4a07d1c9218 Mon Sep 17 00:00:00 2001 From: vatsrahul1001 Date: Thu, 5 Dec 2024 17:18:39 +0530 Subject: [PATCH 1/3] remove deprecations --- .../airflow/providers/slack/hooks/slack.py | 64 +---------- .../slack/notifications/slack_notifier.py | 30 ------ .../providers/slack/operators/slack.py | 24 +---- .../providers/slack/transfers/sql_to_slack.py | 23 +--- .../slack/transfers/sql_to_slack_webhook.py | 31 +----- providers/tests/slack/hooks/test_slack.py | 100 +----------------- providers/tests/slack/operators/test_slack.py | 100 ------------------ .../slack/transfers/test_sql_to_slack.py | 18 +--- .../transfers/test_sql_to_slack_webhook.py | 83 +-------------- 9 files changed, 12 insertions(+), 461 deletions(-) delete mode 100644 providers/src/airflow/providers/slack/notifications/slack_notifier.py diff --git a/providers/src/airflow/providers/slack/hooks/slack.py b/providers/src/airflow/providers/slack/hooks/slack.py index 3f667c9943dcc..63f74d0308b53 100644 --- a/providers/src/airflow/providers/slack/hooks/slack.py +++ b/providers/src/airflow/providers/slack/hooks/slack.py @@ -24,12 +24,11 @@ from pathlib import Path from typing import TYPE_CHECKING, Any, TypedDict -from deprecated import deprecated from slack_sdk import WebClient from slack_sdk.errors import SlackApiError from typing_extensions import NotRequired -from airflow.exceptions import 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 @@ -186,67 +185,6 @@ def call(self, api_method: str, **kwargs) -> SlackResponse: """ return self.client.api_call(api_method, **kwargs) - @deprecated( - reason=( - "This method utilise `files.upload` Slack API method which is being sunset on March 11, 2025. " - "Beginning May 8, 2024, newly-created apps will be unable to 'files.upload' Slack API. " - "Please use `send_file_v2` or `send_file_v1_to_v2` instead." - ), - category=AirflowProviderDeprecationWarning, - ) - def send_file( - self, - *, - channels: str | Sequence[str] | None = None, - file: str | Path | None = None, - content: str | None = None, - filename: str | None = None, - filetype: str | None = None, - initial_comment: str | None = None, - title: str | None = None, - **kwargs, - ) -> SlackResponse: - """ - Create or upload an existing file. - - :param channels: Comma-separated list of channel names or IDs where the file will be shared. - If omitting this parameter, then file will send to workspace. - :param file: Path to file which need to be sent. - :param content: File contents. If omitting this parameter, you must provide a file. - :param filename: Displayed filename. - :param filetype: A file type identifier. - :param initial_comment: The message text introducing the file in specified ``channels``. - :param title: Title of file. - - .. seealso:: - - `Slack API files.upload method `_ - - `File types `_ - """ - if not exactly_one(file, content): - raise ValueError("Either `file` or `content` must be provided, not both.") - elif file: - file = Path(file) - with open(file, "rb") as fp: - if not filename: - filename = file.name - return self.client.files_upload( - file=fp, - filename=filename, - filetype=filetype, - initial_comment=initial_comment, - title=title, - channels=channels, - ) - - return self.client.files_upload( - content=content, - filename=filename, - filetype=filetype, - initial_comment=initial_comment, - title=title, - channels=channels, - ) - def send_file_v2( self, *, diff --git a/providers/src/airflow/providers/slack/notifications/slack_notifier.py b/providers/src/airflow/providers/slack/notifications/slack_notifier.py deleted file mode 100644 index 03483ceee0f41..0000000000000 --- a/providers/src/airflow/providers/slack/notifications/slack_notifier.py +++ /dev/null @@ -1,30 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -"""This module is deprecated. Please use :mod:`airflow.providers.slack.notifications.slack`.""" - -from __future__ import annotations - -import warnings - -from airflow.exceptions import AirflowProviderDeprecationWarning -from airflow.providers.slack.notifications.slack import SlackNotifier # noqa: F401 - -warnings.warn( - "This module is deprecated. Please use `airflow.providers.slack.notifications.slack`", - AirflowProviderDeprecationWarning, - stacklevel=2, -) diff --git a/providers/src/airflow/providers/slack/operators/slack.py b/providers/src/airflow/providers/slack/operators/slack.py index a4a31a8527260..440ec377fdc47 100644 --- a/providers/src/airflow/providers/slack/operators/slack.py +++ b/providers/src/airflow/providers/slack/operators/slack.py @@ -18,17 +18,14 @@ from __future__ import annotations import json -import warnings from collections.abc import Sequence from functools import cached_property from typing import TYPE_CHECKING, Any from typing_extensions import Literal -from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.models import BaseOperator from airflow.providers.slack.hooks.slack import SlackHook -from airflow.utils.types import NOTSET, ArgNotSet if TYPE_CHECKING: from slack_sdk.http_retry import RetryHandler @@ -55,7 +52,7 @@ def __init__( self, *, slack_conn_id: str = SlackHook.default_conn_name, - method: str | None = None, + method: str, api_params: dict | None = None, base_url: str | None = None, proxy: str | None = None, @@ -63,13 +60,6 @@ def __init__( retry_handlers: list[RetryHandler] | None = None, **kwargs, ) -> None: - if not method: - warnings.warn( - "Define `method` parameter as empty string or None is deprecated. " - "In the future it will raise an error on task initialisation.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) super().__init__(**kwargs) self.slack_conn_id = slack_conn_id self.method = method @@ -234,21 +224,9 @@ def __init__( content: str | None = None, title: str | None = None, method_version: Literal["v1", "v2"] = "v2", - channel: str | Sequence[str] | None | ArgNotSet = NOTSET, snippet_type: str | None = None, **kwargs, ) -> None: - if channel is not NOTSET: - warnings.warn( - "Argument `channel` is deprecated and will removed in a future releases. " - "Please use `channels` instead.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - if channels: - raise ValueError(f"Cannot set both arguments: channel={channel!r} and channels={channels!r}.") - channels = channel # type: ignore[assignment] - super().__init__(method="files.upload", **kwargs) self.channels = channels self.initial_comment = initial_comment diff --git a/providers/src/airflow/providers/slack/transfers/sql_to_slack.py b/providers/src/airflow/providers/slack/transfers/sql_to_slack.py index 91d2237a1194b..2ad3b9f341ac9 100644 --- a/providers/src/airflow/providers/slack/transfers/sql_to_slack.py +++ b/providers/src/airflow/providers/slack/transfers/sql_to_slack.py @@ -21,13 +21,11 @@ from tempfile import NamedTemporaryFile from typing import TYPE_CHECKING, Any -from deprecated import deprecated from typing_extensions import Literal -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException +from airflow.exceptions import AirflowException, AirflowSkipException from airflow.providers.slack.hooks.slack import SlackHook from airflow.providers.slack.transfers.base_sql_to_slack import BaseSqlToSlackOperator -from airflow.providers.slack.transfers.sql_to_slack_webhook import SqlToSlackWebhookOperator from airflow.providers.slack.utils import parse_filename if TYPE_CHECKING: @@ -169,22 +167,3 @@ def execute(self, context: Context) -> None: initial_comment=self.slack_initial_comment, title=self.slack_title, ) - - -@deprecated( - reason=( - "`airflow.providers.slack.transfers.sql_to_slack.SqlToSlackOperator` has been renamed " - "and moved `airflow.providers.slack.transfers.sql_to_slack_webhook.SqlToSlackWebhookOperator` " - "this operator deprecated and will be removed in future" - ), - category=AirflowProviderDeprecationWarning, -) -class SqlToSlackOperator(SqlToSlackWebhookOperator): - """ - Executes an SQL statement in a given SQL connection and sends the results to Slack Incoming Webhook. - - Deprecated, use :class:`airflow.providers.slack.transfers.sql_to_slack_webhook.SqlToSlackWebhookOperator` - """ - - def __init__(self, *args, **kwargs): - super().__init__(*args, **kwargs) diff --git a/providers/src/airflow/providers/slack/transfers/sql_to_slack_webhook.py b/providers/src/airflow/providers/slack/transfers/sql_to_slack_webhook.py index cfc3944e228cd..4861a778ea7a3 100644 --- a/providers/src/airflow/providers/slack/transfers/sql_to_slack_webhook.py +++ b/providers/src/airflow/providers/slack/transfers/sql_to_slack_webhook.py @@ -16,17 +16,14 @@ # under the License. from __future__ import annotations -import warnings from collections.abc import Iterable, Mapping, Sequence from typing import TYPE_CHECKING, Any -from deprecated import deprecated from tabulate import tabulate -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.providers.slack.hooks.slack_webhook import SlackWebhookHook from airflow.providers.slack.transfers.base_sql_to_slack import BaseSqlToSlackOperator -from airflow.utils.types import NOTSET, ArgNotSet if TYPE_CHECKING: from airflow.utils.context import Context @@ -89,23 +86,8 @@ def __init__( slack_message: str, results_df_name: str = "results_df", parameters: list | tuple | Mapping[str, Any] | None = None, - slack_conn_id: str | ArgNotSet = NOTSET, **kwargs, ) -> None: - if slack_conn_id is not NOTSET: - warnings.warn( - "Parameter `slack_conn_id` is deprecated because this attribute initially intend to use with " - "Slack API however this operator provided integration with Slack Incoming Webhook. " - "Please use `slack_webhook_conn_id` instead.", - AirflowProviderDeprecationWarning, - stacklevel=3, - ) - if slack_webhook_conn_id and slack_conn_id != slack_webhook_conn_id: - raise ValueError( - "Conflicting Connection ids provided, " - f"slack_webhook_conn_id={slack_webhook_conn_id!r}, slack_conn_id={slack_conn_id!r}." - ) - slack_webhook_conn_id = slack_conn_id # type: ignore[assignment] if not slack_webhook_conn_id: raise ValueError("Got an empty `slack_webhook_conn_id` value.") super().__init__( @@ -164,14 +146,3 @@ def execute(self, context: Context) -> None: self._render_and_send_slack_message(context, df) self.log.debug("Finished sending SQL data to Slack") - - @property - @deprecated( - reason=( - "`SqlToSlackWebhookOperator.slack_conn_id` property deprecated and will be removed in a future. " - "Please use `slack_webhook_conn_id` instead." - ), - category=AirflowProviderDeprecationWarning, - ) - def slack_conn_id(self): - return self.slack_webhook_conn_id diff --git a/providers/tests/slack/hooks/test_slack.py b/providers/tests/slack/hooks/test_slack.py index 10e5a094e9d8f..73c44f426de3b 100644 --- a/providers/tests/slack/hooks/test_slack.py +++ b/providers/tests/slack/hooks/test_slack.py @@ -26,7 +26,7 @@ from slack_sdk.http_retry.builtin_handlers import ConnectionErrorRetryHandler, RateLimitErrorRetryHandler from slack_sdk.web.slack_response import SlackResponse -from airflow.exceptions import AirflowNotFoundException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowNotFoundException from airflow.models.connection import Connection from airflow.providers.slack.hooks.slack import SlackHook @@ -297,104 +297,6 @@ def test_hook_connection_failed(self, mocked_client, response_data): mocked_client.api_call.assert_called_once_with("auth.test") assert not conn_test[0] - @pytest.mark.parametrize( - "file,content", - [ - pytest.param(None, None, id="both-none"), - pytest.param("", "", id="both-empty"), - pytest.param("foo.bar", "test-content", id="both-specified"), - ], - ) - def test_send_file_wrong_parameters(self, file, content): - hook = SlackHook(slack_conn_id=SLACK_API_DEFAULT_CONN_ID) - warning_message = "use `send_file_v2` or `send_file_v1_to_v2" - error_message = r"Either `file` or `content` must be provided, not both\." - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - with pytest.raises(ValueError, match=error_message): - hook.send_file(file=file, content=content) - - @pytest.mark.parametrize("initial_comment", [None, "test comment"]) - @pytest.mark.parametrize("title", [None, "test title"]) - @pytest.mark.parametrize("filetype", [None, "auto"]) - @pytest.mark.parametrize("channels", [None, "#random", "#random,#general", ("#random", "#general")]) - def test_send_file_path( - self, mocked_client, tmp_path_factory, initial_comment, title, filetype, channels - ): - """Test send file by providing filepath.""" - tmp = tmp_path_factory.mktemp("test_send_file_path") - file = tmp / "test.json" - file.write_text('{"foo": "bar"}') - - hook = SlackHook(slack_conn_id=SLACK_API_DEFAULT_CONN_ID) - warning_message = "use `send_file_v2` or `send_file_v1_to_v2" - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - hook.send_file( - channels=channels, - file=file, - filename="filename.mock", - initial_comment=initial_comment, - title=title, - filetype=filetype, - ) - - mocked_client.files_upload.assert_called_once_with( - channels=channels, - file=mock.ANY, # Validate file properties later - filename="filename.mock", - initial_comment=initial_comment, - title=title, - filetype=filetype, - ) - - # Validate file properties - mock_file = mocked_client.files_upload.call_args.kwargs["file"] - assert mock_file.mode == "rb" - assert mock_file.name == str(file) - - @pytest.mark.parametrize("filename", ["test.json", "1.parquet.snappy"]) - def test_send_file_path_set_filename(self, mocked_client, tmp_path_factory, filename): - """Test set filename in send_file method if it not set.""" - tmp = tmp_path_factory.mktemp("test_send_file_path_set_filename") - file = tmp / filename - file.write_text('{"foo": "bar"}') - - hook = SlackHook(slack_conn_id=SLACK_API_DEFAULT_CONN_ID) - warning_message = "use `send_file_v2` or `send_file_v1_to_v2" - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - hook.send_file(file=file) - - assert mocked_client.files_upload.call_count == 1 - call_args = mocked_client.files_upload.call_args.kwargs - assert "filename" in call_args - assert call_args["filename"] == filename - - @pytest.mark.parametrize("initial_comment", [None, "test comment"]) - @pytest.mark.parametrize("title", [None, "test title"]) - @pytest.mark.parametrize("filetype", [None, "auto"]) - @pytest.mark.parametrize("filename", [None, "foo.bar"]) - @pytest.mark.parametrize("channels", [None, "#random", "#random,#general", ("#random", "#general")]) - def test_send_file_content(self, mocked_client, initial_comment, title, filetype, channels, filename): - """Test send file by providing content.""" - hook = SlackHook(slack_conn_id=SLACK_API_DEFAULT_CONN_ID) - warning_message = "use `send_file_v2` or `send_file_v1_to_v2" - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - hook.send_file( - channels=channels, - content='{"foo": "bar"}', - filename=filename, - initial_comment=initial_comment, - title=title, - filetype=filetype, - ) - mocked_client.files_upload.assert_called_once_with( - channels=channels, - content='{"foo": "bar"}', - filename=filename, - initial_comment=initial_comment, - title=title, - filetype=filetype, - ) - @pytest.mark.parametrize( "uri", [ diff --git a/providers/tests/slack/operators/test_slack.py b/providers/tests/slack/operators/test_slack.py index 7f2d316387b2a..d49bb7e19661b 100644 --- a/providers/tests/slack/operators/test_slack.py +++ b/providers/tests/slack/operators/test_slack.py @@ -22,13 +22,11 @@ import pytest -from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.providers.slack.operators.slack import ( SlackAPIFileOperator, SlackAPIOperator, SlackAPIPostOperator, ) -from airflow.utils.task_instance_session import set_current_task_instance_session SLACK_API_TEST_CONNECTION_ID = "test_slack_conn_id" DEFAULT_HOOKS_PARAMETERS = {"base_url": None, "timeout": None, "proxy": None, "retry_handlers": None} @@ -72,20 +70,6 @@ def test_hook(self, mock_slack_hook_cls, slack_op_kwargs, hook_extra_kwargs): slack_conn_id=SLACK_API_TEST_CONNECTION_ID, **hook_extra_kwargs ) - @pytest.mark.parametrize("slack_method", [pytest.param("", id="empty"), pytest.param(None, id="none")]) - def test_empty_method(self, slack_method): - warning_message = "Define `method` parameter as empty string or None is deprecated" - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - # Should only raise a warning on task initialisation - op = SlackAPIOperator( - task_id="test-mask-token", - slack_conn_id=SLACK_API_TEST_CONNECTION_ID, - method=slack_method, - ) - - with pytest.raises(ValueError, match="Expected non empty `method` attribute"): - op.execute({}) - class TestSlackAPIPostOperator: def setup_method(self): @@ -236,7 +220,6 @@ def test_init_with_valid_params(self): @pytest.mark.parametrize( "method_version, method_name", [ - pytest.param("v1", "send_file", id="v1"), pytest.param("v2", "send_file_v1_to_v2", id="v2"), ], ) @@ -271,7 +254,6 @@ def test_api_call_params_with_content_args( @pytest.mark.parametrize( "method_version, method_name", [ - pytest.param("v1", "send_file", id="v1"), pytest.param("v2", "send_file_v1_to_v2", id="v2"), ], ) @@ -300,85 +282,3 @@ def test_api_call_params_with_file_args( title=title, snippet_type=snippet_type, ) - - def test_channel_deprecated(self): - warning_message = ( - r"Argument `channel` is deprecated and will removed in a future releases\. " - r"Please use `channels` instead\." - ) - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - op = SlackAPIFileOperator( - task_id="slack", - slack_conn_id=SLACK_API_TEST_CONNECTION_ID, - channel="#random", - channels=None, - ) - assert op.channels == "#random" - - def test_both_channel_and_channels_set(self): - error_message = r"Cannot set both arguments: channel=.* and channels=.*\." - warning_message = ( - r"Argument `channel` is deprecated and will removed in a future releases\. " - r"Please use `channels` instead\." - ) - with pytest.raises(ValueError, match=error_message): - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_message): - SlackAPIFileOperator( - task_id="slack", - slack_conn_id=SLACK_API_TEST_CONNECTION_ID, - channel="#random", - channels="#general", - ) - - @pytest.mark.db_test - @pytest.mark.parametrize( - "channel", - [ - pytest.param("#contributors", id="single-channel"), - pytest.param(["#random", "#general"], id="multiple-channels"), - ], - ) - def test_partial_deprecated_channel(self, channel, dag_maker, session): - with dag_maker(dag_id="test_partial_deprecated_channel", session=session): - SlackAPIFileOperator.partial( - task_id="fake-task-id", - slack_conn_id="fake-conn-id", - channel=channel, - ).expand(filename=["/dev/zero", "/dev/urandom"]) - - dr = dag_maker.create_dagrun() - tis = dr.get_task_instances(session=session) - with set_current_task_instance_session(session=session): - warning_match = r"Argument `channel` is deprecated.*use `channels` instead" - for ti in tis: - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_match): - ti.render_templates() - assert ti.task.channels == channel - - @pytest.mark.db_test - @pytest.mark.parametrize( - "channel, channels", - [ - pytest.param("#contributors", "#user-troubleshooting", id="ambiguous-channel-params"), - pytest.param(["#random", "#general"], ["#random", "#general"], id="non-ambiguous-channel-params"), - ], - ) - def test_partial_both_channel_parameters(self, channel, channels, dag_maker, session): - with dag_maker("test_partial_both_channel_parameters", session=session): - SlackAPIFileOperator.partial( - task_id="fake-task-id", - slack_conn_id="fake-conn-id", - channel=channel, - channels=channels, - ).expand(filename=["/dev/zero", "/dev/urandom"]) - - dr = dag_maker.create_dagrun(session=session) - tis = dr.get_task_instances(session=session) - with set_current_task_instance_session(session=session): - warning_match = r"Argument `channel` is deprecated.*use `channels` instead" - for ti in tis: - with ( - pytest.warns(AirflowProviderDeprecationWarning, match=warning_match), - pytest.raises(ValueError, match="Cannot set both arguments"), - ): - ti.render_templates() diff --git a/providers/tests/slack/transfers/test_sql_to_slack.py b/providers/tests/slack/transfers/test_sql_to_slack.py index a71833f6fb353..cdf6ee41122b2 100644 --- a/providers/tests/slack/transfers/test_sql_to_slack.py +++ b/providers/tests/slack/transfers/test_sql_to_slack.py @@ -20,8 +20,8 @@ import pytest -from airflow.exceptions import AirflowProviderDeprecationWarning, AirflowSkipException -from airflow.providers.slack.transfers.sql_to_slack import SqlToSlackApiFileOperator, SqlToSlackOperator +from airflow.exceptions import AirflowSkipException +from airflow.providers.slack.transfers.sql_to_slack import SqlToSlackApiFileOperator from airflow.utils import timezone TEST_DAG_ID = "sql_to_slack_unit_test" @@ -223,17 +223,3 @@ def test_null_output_raise_error(self, mock_get_query_results, mock_slack_hook_c with pytest.raises(ValueError, match=r"output df must be non-empty\. Failing"): op.execute(mock.MagicMock()) mock_slack_hook_cls.assert_not_called() - - -def test_deprecated_sql_to_slack_operator(): - warning_pattern = "SqlToSlackOperator` has been renamed and moved" - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_pattern): - SqlToSlackOperator( - task_id="deprecated-sql-to-slack", - sql="SELECT 1", - sql_conn_id="test-sql-conn-id", - slack_webhook_conn_id="test-slack-conn-id", - sql_hook_params=None, - parameters=None, - slack_message="foo-bar", - ) diff --git a/providers/tests/slack/transfers/test_sql_to_slack_webhook.py b/providers/tests/slack/transfers/test_sql_to_slack_webhook.py index c56f895b7bc78..3f4a14fa888b2 100644 --- a/providers/tests/slack/transfers/test_sql_to_slack_webhook.py +++ b/providers/tests/slack/transfers/test_sql_to_slack_webhook.py @@ -16,18 +16,15 @@ # under the License. from __future__ import annotations -from contextlib import nullcontext from unittest import mock import pandas as pd import pytest from airflow import DAG -from airflow.exceptions import AirflowProviderDeprecationWarning from airflow.models import Connection from airflow.providers.slack.transfers.sql_to_slack_webhook import SqlToSlackWebhookOperator from airflow.utils import timezone -from airflow.utils.task_instance_session import set_current_task_instance_session TEST_DAG_ID = "sql_to_slack_unit_test" TEST_TASK_ID = "sql_to_slack_unit_test_task" @@ -128,14 +125,13 @@ def test_rendering_and_message_execution_with_slack_hook(self, mocked_hook): ) @pytest.mark.parametrize( - "slack_webhook_conn_id, slack_conn_id, warning_expected, expected_conn_id", + "slack_webhook_conn_id, warning_expected, expected_conn_id", [ - pytest.param("foo", None, False, "foo", id="slack-webhook-conn-id"), - pytest.param(None, "bar", True, "bar", id="slack-conn-id"), - pytest.param("spam", "spam", True, "spam", id="mixin-conn-ids"), + pytest.param("foo", False, "foo", id="slack-webhook-conn-id"), + pytest.param("spam", True, "spam", id="mixin-conn-ids"), ], ) - def test_resolve_conn_ids(self, slack_webhook_conn_id, slack_conn_id, warning_expected, expected_conn_id): + def test_resolve_conn_ids(self, slack_webhook_conn_id, warning_expected, expected_conn_id): operator_args = { "sql_conn_id": "snowflake_connection", "slack_message": "message: {{ ds }}, {{ xxxx }}", @@ -143,32 +139,10 @@ def test_resolve_conn_ids(self, slack_webhook_conn_id, slack_conn_id, warning_ex } if slack_webhook_conn_id: operator_args["slack_webhook_conn_id"] = slack_webhook_conn_id - if slack_conn_id: - operator_args["slack_conn_id"] = slack_conn_id - ctx = ( - pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `slack_conn_id` is deprecated") - if warning_expected - else nullcontext() - ) - with ctx: - op = self._construct_operator(**operator_args) + op = self._construct_operator(**operator_args) assert op.slack_webhook_conn_id == expected_conn_id - with pytest.warns(AirflowProviderDeprecationWarning, match="slack_conn_id` property deprecated"): - assert op.slack_conn_id == expected_conn_id - - def test_conflicting_conn_id(self): - operator_args = { - "sql_conn_id": "snowflake_connection", - "slack_message": "message: {{ ds }}, {{ xxxx }}", - "sql": "sql {{ ds }}", - } - with ( - pytest.raises(ValueError, match="Conflicting Connection ids provided"), - pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `slack_conn_id` is deprecated"), - ): - self._construct_operator(**operator_args, slack_webhook_conn_id="foo", slack_conn_id="bar") def test_non_existing_slack_webhook_conn_id(self): operator_args = { @@ -271,50 +245,3 @@ def test_hook_params_snowflake(self, mocked_get_connection): assert hook.database == "database" assert hook.role == "role" assert hook.schema == "schema" - - @pytest.mark.parametrize( - "slack_conn_id, slack_webhook_conn_id", - [ - pytest.param("slack_conn_id", None, id="slack-conn-id-only"), - pytest.param("slack_conn_id", "slack_conn_id", id="non-ambiguous-params"), - ], - ) - def test_partial_deprecated_slack_conn_id(self, slack_conn_id, slack_webhook_conn_id, dag_maker, session): - with dag_maker(dag_id="test_partial_deprecated_slack_conn_id", session=session): - SqlToSlackWebhookOperator.partial( - task_id="fake-task-id", - slack_conn_id=slack_conn_id, - slack_webhook_conn_id=slack_webhook_conn_id, - sql_conn_id="fake-sql-conn-id", - slack_message="", - ).expand(sql=["SELECT 1", "SELECT 2"]) - - dr = dag_maker.create_dagrun() - tis = dr.get_task_instances(session=session) - with set_current_task_instance_session(session=session): - warning_match = r"Parameter `slack_conn_id` is deprecated" - for ti in tis: - with pytest.warns(AirflowProviderDeprecationWarning, match=warning_match): - ti.render_templates() - assert ti.task.slack_webhook_conn_id == slack_conn_id - - def test_partial_ambiguous_slack_connections(self, dag_maker, session): - with dag_maker("test_partial_ambiguous_slack_connections", session=session): - SqlToSlackWebhookOperator.partial( - task_id="fake-task-id", - slack_conn_id="slack_conn_id", - slack_webhook_conn_id="slack_webhook_conn_id", - sql_conn_id="fake-sql-conn-id", - slack_message="", - ).expand(sql=["SELECT 1", "SELECT 2"]) - - dr = dag_maker.create_dagrun(session=session) - tis = dr.get_task_instances(session=session) - with set_current_task_instance_session(session=session): - warning_match = r"Parameter `slack_conn_id` is deprecated" - for ti in tis: - with ( - pytest.warns(AirflowProviderDeprecationWarning, match=warning_match), - pytest.raises(ValueError, match="Conflicting Connection ids provided"), - ): - ti.render_templates() From de502df5b71d55b1121f1850747c353f0eee830f Mon Sep 17 00:00:00 2001 From: vatsrahul1001 Date: Thu, 5 Dec 2024 19:20:31 +0530 Subject: [PATCH 2/3] adding changelog --- .../src/airflow/providers/slack/CHANGELOG.rst | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/providers/src/airflow/providers/slack/CHANGELOG.rst b/providers/src/airflow/providers/slack/CHANGELOG.rst index 498595f9a4b70..8b296ca1d5218 100644 --- a/providers/src/airflow/providers/slack/CHANGELOG.rst +++ b/providers/src/airflow/providers/slack/CHANGELOG.rst @@ -27,6 +27,20 @@ Changelog --------- +main +.... + +.. warning:: + All deprecated classes, parameters and features have been removed from the slack provider package. + The following breaking changes were introduced: + + * ``Removed deprecated SqlToSlackOperator. Use SqlToSlackWebhookOperator instead.`` + * ``Removed deprecated send_file method from hooks. Use send_file_v2 or send_file_v1_to_v2 instead.`` + * ``Removed deprecated module lack_notifier.py. Use airflow.providers.slack.notifications.slack instead.`` + * ``Define method parameter as empty string or None is deprecated.`` + * ``Removed deprecated parameter slack_conn_id from SqlToSlackWebhookOperator. Use slack_webhook_conn_id instead.`` + + 8.9.2 ..... From 13238081a1f5e72a3e316c96cf9c34513b923cb6 Mon Sep 17 00:00:00 2001 From: vatsrahul1001 <43964496+vatsrahul1001@users.noreply.github.com> Date: Fri, 6 Dec 2024 21:37:18 +0530 Subject: [PATCH 3/3] Update providers/src/airflow/providers/slack/CHANGELOG.rst Co-authored-by: Wei Lee --- providers/src/airflow/providers/slack/CHANGELOG.rst | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/providers/src/airflow/providers/slack/CHANGELOG.rst b/providers/src/airflow/providers/slack/CHANGELOG.rst index 8b296ca1d5218..522732128ec4e 100644 --- a/providers/src/airflow/providers/slack/CHANGELOG.rst +++ b/providers/src/airflow/providers/slack/CHANGELOG.rst @@ -34,11 +34,11 @@ main All deprecated classes, parameters and features have been removed from the slack provider package. The following breaking changes were introduced: - * ``Removed deprecated SqlToSlackOperator. Use SqlToSlackWebhookOperator instead.`` - * ``Removed deprecated send_file method from hooks. Use send_file_v2 or send_file_v1_to_v2 instead.`` - * ``Removed deprecated module lack_notifier.py. Use airflow.providers.slack.notifications.slack instead.`` - * ``Define method parameter as empty string or None is deprecated.`` - * ``Removed deprecated parameter slack_conn_id from SqlToSlackWebhookOperator. Use slack_webhook_conn_id instead.`` + * Removed deprecated ``SqlToSlackOperator``. Use ``SqlToSlackWebhookOperator`` instead. + * Removed deprecated ``send_file`` method from hooks. Use ``send_file_v2`` or ``send_file_v1_to_v2`` instead. + * Removed deprecated module lack_notifier.py. Use ``airflow.providers.slack.notifications.slack`` instead. + * Define method parameter as empty string or None is deprecated. + * Removed deprecated parameter ``slack_conn_id`` from ``SqlToSlackWebhookOperator``. Use ``slack_webhook_conn_id`` instead. 8.9.2