From df94ec86beae7186d4d724d449e7de850a29468c Mon Sep 17 00:00:00 2001 From: vatsrahul1001 Date: Sat, 7 Dec 2024 00:38:37 +0530 Subject: [PATCH 1/3] remove deprecations --- .../src/airflow/providers/sftp/hooks/sftp.py | 37 +---- .../airflow/providers/sftp/operators/sftp.py | 43 +----- providers/tests/sftp/hooks/test_sftp.py | 41 +---- providers/tests/sftp/operators/test_sftp.py | 141 +++--------------- 4 files changed, 33 insertions(+), 229 deletions(-) diff --git a/providers/src/airflow/providers/sftp/hooks/sftp.py b/providers/src/airflow/providers/sftp/hooks/sftp.py index 1a826cd645c7a..764b3ec68d5ca 100644 --- a/providers/src/airflow/providers/sftp/hooks/sftp.py +++ b/providers/src/airflow/providers/sftp/hooks/sftp.py @@ -22,7 +22,6 @@ import datetime import os import stat -import warnings from collections.abc import Sequence from fnmatch import fnmatch from typing import TYPE_CHECKING, Any, Callable @@ -30,7 +29,7 @@ import asyncssh from asgiref.sync import sync_to_async -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook from airflow.providers.ssh.hooks.ssh import SSHHook @@ -62,7 +61,6 @@ class SFTPHook(SSHHook): For consistency reasons with SSHHook, the preferred parameter is "ssh_conn_id". :param ssh_conn_id: The :ref:`sftp connection id` - :param ssh_hook: Optional SSH hook (included to support passing of an SSH hook to the SFTP operator) """ conn_name_attr = "ssh_conn_id" @@ -82,39 +80,12 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]: def __init__( self, ssh_conn_id: str | None = "sftp_default", - ssh_hook: SSHHook | None = None, host_proxy_cmd: str | None = None, *args, **kwargs, ) -> None: self.conn: paramiko.SFTPClient | None = None - # TODO: remove support for ssh_hook when it is removed from SFTPOperator - self.ssh_hook = ssh_hook - - if self.ssh_hook is not None: - warnings.warn( - "Parameter `ssh_hook` is deprecated and will be removed in a future version.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - if not isinstance(self.ssh_hook, SSHHook): - raise AirflowException( - f"ssh_hook must be an instance of SSHHook, but got {type(self.ssh_hook)}" - ) - self.log.info("ssh_hook is provided. It will be used to generate SFTP connection.") - self.ssh_conn_id = self.ssh_hook.ssh_conn_id - return - - ftp_conn_id = kwargs.pop("ftp_conn_id", None) - if ftp_conn_id: - warnings.warn( - "Parameter `ftp_conn_id` is deprecated. Please use `ssh_conn_id` instead.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - ssh_conn_id = ftp_conn_id - kwargs["ssh_conn_id"] = ssh_conn_id kwargs["host_proxy_cmd"] = host_proxy_cmd self.ssh_conn_id = ssh_conn_id @@ -124,11 +95,7 @@ def __init__( def get_conn(self) -> paramiko.SFTPClient: # type: ignore[override] """Open an SFTP connection to the remote host.""" if self.conn is None: - # TODO: remove support for ssh_hook when it is removed from SFTPOperator - if self.ssh_hook is not None: - self.conn = self.ssh_hook.get_conn().open_sftp() - else: - self.conn = super().get_conn().open_sftp() + self.conn = super().get_conn().open_sftp() return self.conn def close_conn(self) -> None: diff --git a/providers/src/airflow/providers/sftp/operators/sftp.py b/providers/src/airflow/providers/sftp/operators/sftp.py index 95a2880ae2eac..8538f874585f1 100644 --- a/providers/src/airflow/providers/sftp/operators/sftp.py +++ b/providers/src/airflow/providers/sftp/operators/sftp.py @@ -21,17 +21,15 @@ import os import socket -import warnings from collections.abc import Sequence from pathlib import Path from typing import Any import paramiko -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.providers.sftp.hooks.sftp import SFTPHook -from airflow.providers.ssh.hooks.ssh import SSHHook class SFTPOperation: @@ -48,15 +46,12 @@ class SFTPOperator(BaseOperator): This operator uses sftp_hook to open sftp transport channel that serve as basis for file transfer. :param ssh_conn_id: :ref:`ssh connection id` - from airflow Connections. `ssh_conn_id` will be ignored if `ssh_hook` - or `sftp_hook` is provided. + from airflow Connections. :param sftp_hook: predefined SFTPHook to use Either `sftp_hook` or `ssh_conn_id` needs to be provided. - :param ssh_hook: Deprecated - predefined SSHHook to use for remote execution - Use `sftp_hook` instead. :param remote_host: remote host to connect (templated) Nullable. If provided, it will replace the `remote_host` which was - defined in `sftp_hook`/`ssh_hook` or predefined in the connection of `ssh_conn_id`. + defined in `sftp_hook` or predefined in the connection of `ssh_conn_id`. :param local_filepath: local file path or list of local file paths to get or put. (templated) :param remote_filepath: remote file path or list of remote file paths to get or put. (templated) :param operation: specify operation 'get' or 'put', defaults to put @@ -86,7 +81,6 @@ class SFTPOperator(BaseOperator): def __init__( self, *, - ssh_hook: SSHHook | None = None, sftp_hook: SFTPHook | None = None, ssh_conn_id: str | None = None, remote_host: str | None = None, @@ -98,7 +92,6 @@ def __init__( **kwargs, ) -> None: super().__init__(**kwargs) - self.ssh_hook = ssh_hook self.sftp_hook = sftp_hook self.ssh_conn_id = ssh_conn_id self.remote_host = remote_host @@ -131,35 +124,13 @@ def execute(self, context: Any) -> str | list[str] | None: f"expected {SFTPOperation.GET} or {SFTPOperation.PUT}." ) - # TODO: remove support for ssh_hook in next major provider version in hook and operator - if self.ssh_hook is not None and self.sftp_hook is not None: - raise AirflowException( - "Both `ssh_hook` and `sftp_hook` are defined. Please use only one of them." - ) - - if self.ssh_hook is not None: - if not isinstance(self.ssh_hook, SSHHook): - self.log.info("ssh_hook is invalid. Trying ssh_conn_id to create SFTPHook.") - self.sftp_hook = SFTPHook(ssh_conn_id=self.ssh_conn_id) - if self.sftp_hook is None: - warnings.warn( - "Parameter `ssh_hook` is deprecated. " - "Please use `sftp_hook` instead. " - "The old parameter `ssh_hook` will be removed in a future version.", - AirflowProviderDeprecationWarning, - stacklevel=2, - ) - self.sftp_hook = SFTPHook(ssh_hook=self.ssh_hook) - file_msg = None try: if self.ssh_conn_id: if self.sftp_hook and isinstance(self.sftp_hook, SFTPHook): - self.log.info("ssh_conn_id is ignored when sftp_hook/ssh_hook is provided.") + self.log.info("ssh_conn_id is ignored when sftp_hook is provided.") else: - self.log.info( - "sftp_hook/ssh_hook not provided or invalid. Trying ssh_conn_id to create SFTPHook." - ) + self.log.info("sftp_hook not provided or invalid. Trying ssh_conn_id to create SFTPHook.") self.sftp_hook = SFTPHook(ssh_conn_id=self.ssh_conn_id) if not self.sftp_hook: @@ -217,7 +188,7 @@ def get_openlineage_facets_on_start(self): exc_info=True, ) - hook = self.sftp_hook or self.ssh_hook or SFTPHook(ssh_conn_id=self.ssh_conn_id) + hook = self.sftp_hook or SFTPHook(ssh_conn_id=self.ssh_conn_id) if self.remote_host is not None: remote_host = self.remote_host @@ -235,8 +206,6 @@ def get_openlineage_facets_on_start(self): if hasattr(hook, "port"): remote_port = hook.port - elif hasattr(hook, "ssh_hook"): - remote_port = hook.ssh_hook.port # Since v4.1.0, SFTPOperator accepts both a string (single file) and a list of # strings (multiple files) as local_filepath and remote_filepath, and internally diff --git a/providers/tests/sftp/hooks/test_sftp.py b/providers/tests/sftp/hooks/test_sftp.py index 5f2c34a8cc0e6..1ebd45fc9e2fd 100644 --- a/providers/tests/sftp/hooks/test_sftp.py +++ b/providers/tests/sftp/hooks/test_sftp.py @@ -30,10 +30,9 @@ from asyncssh import SFTPAttrs, SFTPNoSuchFile from asyncssh.sftp import SFTPName -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.models import Connection from airflow.providers.sftp.hooks.sftp import SFTPHook, SFTPHookAsync -from airflow.providers.ssh.hooks.ssh import SSHHook from airflow.utils.session import provide_session pytestmark = pytest.mark.db_test @@ -383,44 +382,6 @@ def test_connection_success(self, mock_get_connection): assert status is True assert msg == "Connection successfully tested" - @mock.patch("airflow.providers.sftp.hooks.sftp.SFTPHook.get_connection") - def test_deprecation_ftp_conn_id(self, mock_get_connection): - connection = Connection(conn_id="ftp_default", login="login", host="host") - mock_get_connection.return_value = connection - # If `ftp_conn_id` is provided, it will be used but would show a deprecation warning. - with pytest.warns(AirflowProviderDeprecationWarning, match=r"Parameter `ftp_conn_id` is deprecated"): - assert SFTPHook(ftp_conn_id="ftp_default").ssh_conn_id == "ftp_default" - - # If both are provided, ftp_conn_id will be used but would show a deprecation warning. - with pytest.warns(AirflowProviderDeprecationWarning, match=r"Parameter `ftp_conn_id` is deprecated"): - assert ( - SFTPHook(ftp_conn_id="ftp_default", ssh_conn_id="sftp_default").ssh_conn_id == "ftp_default" - ) - - # If `ssh_conn_id` is provided, it should use it for ssh_conn_id - assert SFTPHook(ssh_conn_id="sftp_default").ssh_conn_id == "sftp_default" - # Default is 'sftp_default - assert SFTPHook().ssh_conn_id == "sftp_default" - - @mock.patch("airflow.providers.sftp.hooks.sftp.SFTPHook.get_connection") - def test_invalid_ssh_hook(self, mock_get_connection): - connection = Connection(conn_id="sftp_default", login="root", host="localhost") - mock_get_connection.return_value = connection - with ( - pytest.raises(AirflowException, match="ssh_hook must be an instance of SSHHook"), - pytest.warns(AirflowProviderDeprecationWarning, match=r"Parameter `ssh_hook` is deprecated.*"), - ): - SFTPHook(ssh_hook="invalid_hook") - - @mock.patch("airflow.providers.ssh.hooks.ssh.SSHHook.get_connection") - def test_valid_ssh_hook(self, mock_get_connection): - connection = Connection(conn_id="sftp_test", login="root", host="localhost") - mock_get_connection.return_value = connection - with pytest.warns(AirflowProviderDeprecationWarning, match=r"Parameter `ssh_hook` is deprecated.*"): - hook = SFTPHook(ssh_hook=SSHHook(ssh_conn_id="sftp_test")) - assert hook.ssh_conn_id == "sftp_test" - assert isinstance(hook.get_conn(), paramiko.SFTPClient) - def test_get_suffix_pattern_match(self): output = self.hook.get_file_by_pattern(self.temp_dir, "*.txt") # In CI files might have different name, so we check that file found rather than actual name diff --git a/providers/tests/sftp/operators/test_sftp.py b/providers/tests/sftp/operators/test_sftp.py index 2bd4be3d269e6..86965e0ed1913 100644 --- a/providers/tests/sftp/operators/test_sftp.py +++ b/providers/tests/sftp/operators/test_sftp.py @@ -26,7 +26,7 @@ import paramiko import pytest -from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning +from airflow.exceptions import AirflowException from airflow.models import DAG, Connection from airflow.providers.common.compat.openlineage.facet import Dataset from airflow.providers.sftp.hooks.sftp import SFTPHook @@ -110,7 +110,7 @@ def test_pickle_file_transfer_put(self, dag_maker): with dag_maker(dag_id="unit_tests_sftp_op_pickle_file_transfer_put", start_date=DEFAULT_DATE): SFTPOperator( # Put test file to remote. task_id="put_test_task", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.PUT, @@ -124,9 +124,8 @@ def test_pickle_file_transfer_put(self, dag_maker): ) tis = {ti.task_id: ti for ti in dag_maker.create_dagrun().task_instances} - with pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*"): - tis["put_test_task"].run() - tis["check_file_task"].run() + tis["put_test_task"].run() + tis["check_file_task"].run() pulled = tis["check_file_task"].xcom_pull(task_ids="check_file_task", key="return_value") assert pulled.strip() == test_local_file_content @@ -148,7 +147,7 @@ def test_file_transfer_no_intermediate_dir_error_put(self, create_task_instance_ SFTPOperator, dag_id="unit_tests_sftp_op_file_transfer_no_intermediate_dir_error_put", task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath_int_dir, operation=SFTPOperation.PUT, @@ -156,7 +155,6 @@ def test_file_transfer_no_intermediate_dir_error_put(self, create_task_instance_ ) with ( pytest.raises(AirflowException) as ctx, - pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*"), ): ti2.run() assert "No such file" in str(ctx.value) @@ -175,7 +173,7 @@ def test_file_transfer_with_intermediate_dir_put(self, dag_maker): with dag_maker(dag_id="unit_tests_sftp_op_file_transfer_with_intermediate_dir_put"): SFTPOperator( # Put test file to remote. task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath_int_dir, operation=SFTPOperation.PUT, @@ -189,10 +187,8 @@ def test_file_transfer_with_intermediate_dir_put(self, dag_maker): ) dagrun = dag_maker.create_dagrun(logical_date=timezone.utcnow()) tis = {ti.task_id: ti for ti in dagrun.task_instances} - with pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*"): - tis["test_sftp"].run() - tis["test_check_file"].run() - + tis["test_sftp"].run() + tis["test_check_file"].run() pulled = tis["test_check_file"].xcom_pull(task_ids="test_check_file", key="return_value") assert pulled.strip() == test_local_file_content @@ -209,7 +205,7 @@ def test_json_file_transfer_put(self, dag_maker): with dag_maker(dag_id="unit_tests_sftp_op_json_file_transfer_put"): SFTPOperator( # Put test file to remote. task_id="put_test_task", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.PUT, @@ -222,9 +218,9 @@ def test_json_file_transfer_put(self, dag_maker): ) dagrun = dag_maker.create_dagrun(logical_date=timezone.utcnow()) tis = {ti.task_id: ti for ti in dagrun.task_instances} - with pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*"): - tis["put_test_task"].run() - tis["check_file_task"].run() + + tis["put_test_task"].run() + tis["check_file_task"].run() pulled = tis["check_file_task"].xcom_pull(task_ids="check_file_task", key="return_value") assert pulled.strip() == b64encode(test_local_file_content).decode("utf-8") @@ -242,16 +238,13 @@ def test_pickle_file_transfer_get(self, dag_maker, create_remote_file_and_cleanu with dag_maker(dag_id="unit_tests_sftp_op_pickle_file_transfer_get"): SFTPOperator( # Get remote file to local. task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.GET, ) for ti in dag_maker.create_dagrun(logical_date=timezone.utcnow()).task_instances: - with pytest.warns( - AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*" - ): - ti.run() + ti.run() # Test the received content. with open(self.test_local_filepath, "rb") as file: @@ -263,16 +256,13 @@ def test_json_file_transfer_get(self, dag_maker, create_remote_file_and_cleanup) with dag_maker(dag_id="unit_tests_sftp_op_json_file_transfer_get"): SFTPOperator( # Get remote file to local. task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.GET, ) for ti in dag_maker.create_dagrun(logical_date=timezone.utcnow()).task_instances: - with pytest.warns( - AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*" - ): - ti.run() + ti.run() # Test the received content. content_received = None @@ -286,7 +276,7 @@ def test_file_transfer_no_intermediate_dir_error_get(self, dag_maker, create_rem with dag_maker(dag_id="unit_tests_sftp_op_file_transfer_no_intermediate_dir_error_get"): SFTPOperator( # Try to GET test file from remote. task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath_int_dir, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.GET, @@ -297,9 +287,6 @@ def test_file_transfer_no_intermediate_dir_error_get(self, dag_maker, create_rem # does not exist. with ( pytest.raises(AirflowException) as ctx, - pytest.warns( - AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*" - ), ): ti.run() assert "No such file" in str(ctx.value) @@ -310,7 +297,7 @@ def test_file_transfer_with_intermediate_dir_error_get(self, dag_maker, create_r with dag_maker(dag_id="unit_tests_sftp_op_file_transfer_with_intermediate_dir_error_get"): SFTPOperator( # Get remote file to local. task_id="test_sftp", - ssh_hook=self.hook, + sftp_hook=self.sftp_hook, local_filepath=self.test_local_filepath_int_dir, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.GET, @@ -318,10 +305,7 @@ def test_file_transfer_with_intermediate_dir_error_get(self, dag_maker, create_r ) for ti in dag_maker.create_dagrun(logical_date=timezone.utcnow()).task_instances: - with pytest.warns( - AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*" - ): - ti.run() + ti.run() # Test the received content. content_received = None @@ -336,7 +320,7 @@ def test_arg_checking(self): schedule=None, default_args={"start_date": DEFAULT_DATE}, ) - # Exception should be raised if neither ssh_hook nor ssh_conn_id is provided + # Exception should be raised if ssh_conn_id is not provided task_0 = SFTPOperator( task_id="test_sftp_0", local_filepath=self.test_local_filepath, @@ -347,10 +331,9 @@ def test_arg_checking(self): with pytest.raises(AirflowException, match="Cannot operate without sftp_hook or ssh_conn_id."): task_0.execute(None) - # if ssh_hook is invalid/not provided, use ssh_conn_id to create SSHHook + # use ssh_conn_id to create SSHHook task_1 = SFTPOperator( task_id="test_sftp_1", - ssh_hook="string_rather_than_SSHHook", # type: ignore ssh_conn_id=TEST_CONN_ID, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, @@ -363,7 +346,7 @@ def test_arg_checking(self): task_2 = SFTPOperator( task_id="test_sftp_2", - ssh_conn_id=TEST_CONN_ID, # no ssh_hook provided + ssh_conn_id=TEST_CONN_ID, local_filepath=self.test_local_filepath, remote_filepath=self.test_remote_filepath, operation=SFTPOperation.PUT, @@ -373,53 +356,8 @@ def test_arg_checking(self): task_2.execute(None) assert task_2.sftp_hook.ssh_conn_id == TEST_CONN_ID - # if both valid ssh_hook and ssh_conn_id are provided, ignore ssh_conn_id task_3 = SFTPOperator( task_id="test_sftp_3", - ssh_hook=self.hook, - ssh_conn_id=TEST_CONN_ID, - local_filepath=self.test_local_filepath, - remote_filepath=self.test_remote_filepath, - operation=SFTPOperation.PUT, - dag=dag, - ) - with ( - contextlib.suppress(Exception), - pytest.warns(AirflowProviderDeprecationWarning, match="Parameter `ssh_hook` is deprecated..*"), - ): - task_3.execute(None) - assert task_3.sftp_hook.ssh_conn_id == self.hook.ssh_conn_id - - task_4 = SFTPOperator( - task_id="test_sftp_4", - local_filepath=self.test_local_filepath, - remote_filepath=self.test_remote_filepath, - operation="invalid_operation", - dag=dag, - ) - # Exception should be raised if operation is invalid - with pytest.raises(TypeError, match="Unsupported operation value invalid_operation, "): - task_4.execute(None) - - task_5 = SFTPOperator( - task_id="test_sftp_5", - ssh_hook=self.hook, - sftp_hook=SFTPHook(), - local_filepath=self.test_local_filepath, - remote_filepath=self.test_remote_filepath, - operation=SFTPOperation.PUT, - dag=dag, - ) - - # Exception should be raised if both ssh_hook and sftp_hook are provided - with pytest.raises( - AirflowException, - match="Both `ssh_hook` and `sftp_hook` are defined. Please use only one of them.", - ): - task_5.execute(None) - - task_6 = SFTPOperator( - task_id="test_sftp_6", ssh_conn_id=TEST_CONN_ID, remote_host="remotehost", local_filepath=self.test_local_filepath, @@ -428,8 +366,8 @@ def test_arg_checking(self): dag=dag, ) with contextlib.suppress(Exception): - task_6.execute(None) - assert task_6.sftp_hook.remote_host == "remotehost" + task_3.execute(None) + assert task_3.sftp_hook.remote_host == "remotehost" def test_unequal_local_remote_file_paths(self): with pytest.raises(ValueError): @@ -581,34 +519,3 @@ def test_extract_sftp_hook(self, get_connection, get_conn, operation, expected): assert lineage.inputs == expected[0] assert lineage.outputs == expected[1] - - @pytest.mark.parametrize( - "operation, expected", - TEST_GET_PUT_PARAMS, - ) - @mock.patch("airflow.providers.ssh.hooks.ssh.SSHHook.get_conn", spec=paramiko.SSHClient) - @mock.patch("airflow.providers.ssh.hooks.ssh.SSHHook.get_connection", spec=Connection) - def test_extract_ssh_hook(self, get_connection, get_conn, operation, expected): - get_connection.return_value = Connection( - conn_id="sftp_conn_id", - conn_type="sftp", - host="remotehost", - port=22, - ) - - dag_id = "sftp_dag" - task_id = "sftp_task" - - task = SFTPOperator( - task_id=task_id, - ssh_hook=SSHHook(ssh_conn_id="sftp_conn_id"), - dag=DAG(dag_id, schedule=None), - start_date=timezone.utcnow(), - local_filepath="/path/local", - remote_filepath="/path/remote", - operation=operation, - ) - lineage = task.get_openlineage_facets_on_start() - - assert lineage.inputs == expected[0] - assert lineage.outputs == expected[1] From 584cc5e9bfb7c553b5cccc56a48c811e5c8e0d77 Mon Sep 17 00:00:00 2001 From: vatsrahul1001 Date: Sat, 7 Dec 2024 00:48:57 +0530 Subject: [PATCH 2/3] add changelog --- providers/src/airflow/providers/sftp/CHANGELOG.rst | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/providers/src/airflow/providers/sftp/CHANGELOG.rst b/providers/src/airflow/providers/sftp/CHANGELOG.rst index 5e3ad50b1aedb..070e49855cc92 100644 --- a/providers/src/airflow/providers/sftp/CHANGELOG.rst +++ b/providers/src/airflow/providers/sftp/CHANGELOG.rst @@ -27,6 +27,20 @@ Changelog --------- +main +.... + +.. warning:: + All deprecated classes, parameters and features have been removed from the {provider_name} provider package. + The following breaking changes were introduced: + + * Removed deprecated ``ssh_hook`` parameter from ``SFTPOperator``. Use ``sftp_hook`` instead. + * Removed deprecated ``ssh_hook`` parameter from ``SFTPHook``. + * Removed deprecated ``ftp_conn_id`` parameter from ``SFTPHook``. Use ``ssh_conn_id`` instead. + + + + 4.11.1 ...... From 76a592b4de913c34afcffcdf7e2c25c406064a45 Mon Sep 17 00:00:00 2001 From: vatsrahul1001 Date: Sat, 7 Dec 2024 01:28:46 +0530 Subject: [PATCH 3/3] update provider name in changelog --- providers/src/airflow/providers/sftp/CHANGELOG.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/providers/src/airflow/providers/sftp/CHANGELOG.rst b/providers/src/airflow/providers/sftp/CHANGELOG.rst index 070e49855cc92..2313623f247ed 100644 --- a/providers/src/airflow/providers/sftp/CHANGELOG.rst +++ b/providers/src/airflow/providers/sftp/CHANGELOG.rst @@ -31,7 +31,7 @@ main .... .. warning:: - All deprecated classes, parameters and features have been removed from the {provider_name} provider package. + All deprecated classes, parameters and features have been removed from the sftp provider package. The following breaking changes were introduced: * Removed deprecated ``ssh_hook`` parameter from ``SFTPOperator``. Use ``sftp_hook`` instead.