Skip to content

Commit

Permalink
Remove Provider Deprecations in SSH
Browse files Browse the repository at this point in the history
  • Loading branch information
jscheffl committed Dec 2, 2024
1 parent 33bf3ed commit 2e0a625
Show file tree
Hide file tree
Showing 2 changed files with 3 additions and 88 deletions.
71 changes: 2 additions & 69 deletions providers/src/airflow/providers/ssh/hooks/ssh.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
from __future__ import annotations

import os
import warnings
from base64 import decodebytes
from collections.abc import Sequence
from functools import cached_property
Expand All @@ -29,12 +28,11 @@
from typing import Any

import paramiko
from deprecated import deprecated
from paramiko.config import SSH_PORT
from sshtunnel import SSHTunnelForwarder
from tenacity import Retrying, stop_after_attempt, wait_fixed, wait_random

from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
from airflow.exceptions import AirflowException
from airflow.hooks.base import BaseHook
from airflow.utils.platform import getuser
from airflow.utils.types import NOTSET, ArgNotSet
Expand Down Expand Up @@ -63,8 +61,6 @@ class SSHHook(BaseHook):
:param conn_timeout: timeout (in seconds) for the attempt to connect to the remote_host.
The default is 10 seconds. If provided, it will replace the `conn_timeout` which was
predefined in the connection of `ssh_conn_id`.
:param timeout: (Deprecated). timeout for the attempt to connect to the remote_host.
Use conn_timeout instead.
:param cmd_timeout: timeout (in seconds) for executing the command. The default is 10 seconds.
Nullable, `None` means no timeout. If provided, it will replace the `cmd_timeout`
which was predefined in the connection of `ssh_conn_id`.
Expand Down Expand Up @@ -116,7 +112,6 @@ def __init__(
password: str | None = None,
key_file: str | None = None,
port: int | None = None,
timeout: int | None = None,
conn_timeout: int | None = None,
cmd_timeout: int | ArgNotSet | None = NOTSET,
keepalive_interval: int = 30,
Expand All @@ -133,7 +128,6 @@ def __init__(
self.key_file = key_file
self.pkey = None
self.port = port
self.timeout = timeout
self.conn_timeout = conn_timeout
self.cmd_timeout = cmd_timeout
self.keepalive_interval = keepalive_interval
Expand All @@ -150,7 +144,7 @@ def __init__(
self.host_key = None
self.look_for_keys = True

# Placeholder for deprecated __enter__
# Placeholder for future cached connection
self.client: paramiko.SSHClient | None = None

# Use connection to override defaults
Expand All @@ -175,16 +169,6 @@ def __init__(
if private_key:
self.pkey = self._pkey_from_private_key(private_key, passphrase=private_key_passphrase)

if "timeout" in extra_options:
warnings.warn(
"Extra option `timeout` is deprecated."
"Please use `conn_timeout` instead."
"The old option `timeout` will be removed in a future version.",
category=AirflowProviderDeprecationWarning,
stacklevel=2,
)
self.timeout = int(extra_options["timeout"])

if "conn_timeout" in extra_options and self.conn_timeout is None:
self.conn_timeout = int(extra_options["conn_timeout"])

Expand Down Expand Up @@ -235,18 +219,6 @@ def __init__(
self.host_key = key_constructor(data=decoded_host_key)
self.no_host_key_check = False

if self.timeout:
warnings.warn(
"Parameter `timeout` is deprecated."
"Please use `conn_timeout` instead."
"The old option `timeout` will be removed in a future version.",
category=AirflowProviderDeprecationWarning,
stacklevel=2,
)

if self.conn_timeout is None:
self.conn_timeout = self.timeout if self.timeout else TIMEOUT_DEFAULT

if self.cmd_timeout is NOTSET:
self.cmd_timeout = CMD_TIMEOUT

Expand Down Expand Up @@ -379,24 +351,6 @@ def log_before_sleep(retry_state):
self.client = client
return client

@deprecated(
reason=(
"The contextmanager of SSHHook is deprecated."
"Please use get_conn() as a contextmanager instead."
"This method will be removed in Airflow 2.0"
),
category=AirflowProviderDeprecationWarning,
)
def __enter__(self) -> SSHHook:
"""Return an instance of SSHHook when the `with` statement is used."""
return self

def __exit__(self, exc_type, exc_val, exc_tb) -> None:
"""Clear ssh client after exiting the `with` statement block."""
if self.client is not None:
self.client.close()
self.client = None

def get_tunnel(
self, remote_port: int, remote_host: str = "localhost", local_port: int | None = None
) -> SSHTunnelForwarder:
Expand Down Expand Up @@ -440,27 +394,6 @@ def get_tunnel(

return client

@deprecated(
reason=(
"SSHHook.create_tunnel is deprecated, Please "
"use get_tunnel() instead. But please note that the "
"order of the parameters have changed. "
"This method will be removed in Airflow 2.0"
),
category=AirflowProviderDeprecationWarning,
)
def create_tunnel(
self, local_port: int, remote_port: int, remote_host: str = "localhost"
) -> SSHTunnelForwarder:
"""
Create a tunnel for SSH connection [Deprecated].
:param local_port: local port number
:param remote_port: remote port number
:param remote_host: remote host
"""
return self.get_tunnel(remote_port, remote_host, local_port)

def _pkey_from_private_key(self, private_key: str, passphrase: str | None = None) -> paramiko.PKey:
"""
Create an appropriate Paramiko key for a given private key.
Expand Down
20 changes: 1 addition & 19 deletions providers/src/airflow/providers/ssh/operators/ssh.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,8 @@
from functools import cached_property
from typing import TYPE_CHECKING

from deprecated.classic import deprecated

from airflow.configuration import conf
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowSkipException
from airflow.exceptions import AirflowException, AirflowSkipException
from airflow.models import BaseOperator
from airflow.providers.ssh.hooks.ssh import SSHHook
from airflow.utils.types import NOTSET, ArgNotSet
Expand Down Expand Up @@ -143,26 +141,10 @@ def ssh_hook(self) -> SSHHook:
def hook(self) -> SSHHook:
return self.ssh_hook

@deprecated(reason="use `hook` property instead.", category=AirflowProviderDeprecationWarning)
def get_hook(self) -> SSHHook:
return self.ssh_hook

def get_ssh_client(self) -> SSHClient:
# Remember to use context manager or call .close() on this when done
return self.hook.get_conn()

@deprecated(
reason=(
"exec_ssh_client_command method on SSHOperator is deprecated, call "
"`ssh_hook.exec_ssh_client_command` instead"
),
category=AirflowProviderDeprecationWarning,
)
def exec_ssh_client_command(self, ssh_client: SSHClient, command: str) -> tuple[int, bytes, bytes]:
return self.hook.exec_ssh_client_command(
ssh_client, command, timeout=self.cmd_timeout, environment=self.environment, get_pty=self.get_pty
)

def raise_for_status(self, exit_status: int, stderr: bytes, context=None) -> None:
if context and self.do_xcom_push:
ti = context.get("task_instance")
Expand Down

0 comments on commit 2e0a625

Please sign in to comment.