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 2021/06/30 22:00:00 UTC

[GitHub] [airflow] ecerulm opened a new issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

ecerulm opened a new issue #16738:
URL: https://github.com/apache/airflow/issues/16738


   <!--
   
   Welcome to Apache Airflow!  For a smooth issue process, try to answer the following questions.
   Don't worry if they're not all applicable; just try to include what you can :-)
   
   If you need to include code snippets or logs, please put them in fenced code
   blocks.  If they're super-long, please use the details tag like
   <details><summary>super-long log</summary> lots of stuff </details>
   
   Please delete these comment blocks before submitting the issue.
   
   -->
   
   <!--
   
   IMPORTANT!!!
   
   PLEASE CHECK "SIMILAR TO X EXISTING ISSUES" OPTION IF VISIBLE
   NEXT TO "SUBMIT NEW ISSUE" BUTTON!!!
   
   PLEASE CHECK IF THIS ISSUE HAS BEEN REPORTED PREVIOUSLY USING SEARCH!!!
   
   Please complete the next sections or the issue will be closed.
   These questions are the first thing we need to know to understand the context.
   
   -->
   
   **Apache Airflow version**: 2.0.2
   
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**:
   - **OS** (e.g. from /etc/os-release):
   - **Kernel** (e.g. `uname -a`):
   - **Install tools**:
   - **Others**:
   
   **What happened**:
   
   ```
   ssh-keygen -t rsa -P "" -f test_rsa
   cat test_rsa | python -c 'import sys;import json; print(json.dumps(sys.stdin.read()))' # gives the private key encoded as JSON string to be pasted in the connection extra private_key
   ```
   
   I created an Airflow Connection
   * type: ssh
   * extra:
   ```
   {
   "look_for_keys": "false",
   "no_host_key_check": "true",
   "private_key": "-----BEGIN OPENSSH PRIVATE KEY-----\nb3BlbnNzaC1rZXktdjEAAAAABG5vbmUAAAAEbm9uZQAAAAAAAAABAAABlwAAAAdzc2gtcn\nNhAA........W4tTGFndW5hLU1hY0Jvb2stUHJvAQI=\n-----END OPENSSH PRIVATE KEY-----\n",
   "private_key_passphrase": ""
   }
   ```
   
   When this SSH connection is used in SFTPToS3Operator for example it will incorrectly parse that `private_key` as a `paramiko.dsskey.DSSKey` instead of the correct `paramiko.rsakey.RSAKey`. 
   
   The code responsible for the processing of `private_key` is not **not deterministic** (I don't think `.values()` returns items in any particular order) , but in my case it will always try `paramiko.dsskey.DSSKey`  before it tries `paramiko.rsakey.RSAKey`: 
   
   https://github.com/apache/airflow/blob/8e2a0bc2e39aeaf15b409bbfa8ac0c85aa873815/airflow/providers/ssh/hooks/ssh.py#L363-L369
   
   
   This incorrectly parsed private key will cause a very confusing error  later when it's actually used 
   ```
   [2021-06-30 23:33:14,604] {transport.py:1819} INFO - Connected (version 2.0, client AWS_SFTP_1.0)
   [2021-06-30 23:33:14,732] {transport.py:1819} ERROR - Unknown exception: q must be exactly 160, 224, or 256 bits long
   [2021-06-30 23:33:14,737] {transport.py:1817} ERROR - Traceback (most recent call last):
   [2021-06-30 23:33:14,737] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/paramiko/transport.py", line 2109, in run
   [2021-06-30 23:33:14,737] {transport.py:1817} ERROR -     handler(self.auth_handler, m)
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/paramiko/auth_handler.py", line 298, in _parse_service_accept
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -     sig = self.private_key.sign_ssh_data(blob)
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/paramiko/dsskey.py", line 108, in sign_ssh_data
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -     key = dsa.DSAPrivateNumbers(
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/cryptography/hazmat/primitives/asymmetric/dsa.py", line 244, in private_key
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -     return backend.load_dsa_private_numbers(self)
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/cryptography/hazmat/backends/openssl/backend.py", line 826, in load_dsa_private_numbers
   [2021-06-30 23:33:14,738] {transport.py:1817} ERROR -     dsa._check_dsa_private_numbers(numbers)
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/cryptography/hazmat/primitives/asymmetric/dsa.py", line 282, in _check_dsa_private_numbers
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR -     _check_dsa_parameters(parameters)
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR -   File "/Users/rubelagu/git/apache-airflow-providers-tdh/venv/lib/python3.8/site-packages/cryptography/hazmat/primitives/asymmetric/dsa.py", line 274, in _check_dsa_parameters
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR -     raise ValueError("q must be exactly 160, 224, or 256 bits long")
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR - ValueError: q must be exactly 160, 224, or 256 bits long
   [2021-06-30 23:33:14,739] {transport.py:1817} ERROR - 
   ```
   
   <!-- (please include exact error messages if you can) -->
   
   **What you expected to happen**:
   
   I expected to parse the `private_key` as a RSAKey. 
   
   I did my own test and `paramiko.dsskey.DSSKey.from_private_key(StringIO(private_key), password=passphrase)` will happily parse (incorrectly) a RSA key. The current code assumes that it will raise an exception but it won't. 
   
   <!-- What do you think went wrong? -->
   
   **How to reproduce it**:
   <!---
   
   As minimally and precisely as possible. Keep in mind we do not have access to your cluster or dags.
   
   If you are using kubernetes, please attempt to recreate the issue using minikube or kind.
   
   ## Install minikube/kind
   
   - Minikube https://minikube.sigs.k8s.io/docs/start/
   - Kind https://kind.sigs.k8s.io/docs/user/quick-start/
   
   If this is a UI bug, please provide a screenshot of the bug or a link to a youtube video of the bug in action
   
   You can include images using the .md style of
   ![alt text](http://url/to/img.png)
   
   To record a screencast, mac users can use QuickTime and then create an unlisted youtube video with the resulting .mov file.
   
   --->
   
   
   **Anything else we need to know**:
   
   For me it happens always, I don't think the  order of `.values()` is deterministic, but in my laptop it will always try DSSKey before RSAKey.
   
   
   <!--
   
   How often does this problem occur? Once? Every time etc?
   
   Any relevant logs to include? Put them here in side a detail tag:
   <details><summary>x.log</summary> lots of stuff </details>
   
   -->
   


-- 
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 issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1082002824


   > As it is Airflow 1.10, initiated a discussion. Issue is not allowed for Airflow 1.10.
   > 
   > [https://github.com/apache/airflow/discussions/22589](url)
   
   BTW. Upgrade to Airflow 2. Airfow 1.10 is EOL for more than 10 months now and there were already security fixes in Airflow 2 that will never make it to 1.10 (no more updates there). So I strongly advise you to upgrade.
   
   You've been warned.


-- 
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] anto155 commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
anto155 commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1081994300


   As it is Airflow 1.10, initiated a discussion. Issue is not allowed for Airflow 1.10.
   
   [https://github.com/apache/airflow/discussions/22589](url)


-- 
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 issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-871973743


   Seems it should be easy to fix by      `except paramiko.ssh_exception.SSHException` -> `Exception` - could you please see @ecerulm if thaf fixes the problem (even locally modifying the code) and if so - maybe you could  PR a fix for that? 


-- 
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] ashb closed issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb closed issue #16738:
URL: https://github.com/apache/airflow/issues/16738


   


-- 
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] ashb commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872058448


   Having to specify the key type is "duplication" for most of the cases.
   
   I think we should do two things:
   
   1. Move DSA to the end, so it is tried last (doesn't solve the problem, but it is just so rarely used it shouldn't be first)
   2. Try out the key by calling `ssh_sign_data`
   
   ```python
               try:
                   key = pkey_type.from_private_key(StringIO(private_key), password=passphrase)
                   # Test the key out
                   key.sign_ssh_data(b'')
                   return key
               except (paramiko.ssh_exception.SSHException, ValueError):
                   continue
   ```


-- 
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 closed issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #16738:
URL: https://github.com/apache/airflow/issues/16738


   


-- 
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] ashb commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872051028


   So if DSSKey will happily parse an invalid key there is not _any_ real way around this (other than perhaps DSSKey should be the last as it isn't really recommended to be used anymore)
   
   Do the other key types also happily accept "rubbish", or do they except?


-- 
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] ecerulm commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ecerulm commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872076984


   Please, go ahead


-- 
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] ecerulm commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ecerulm commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872059691


   > Do the other key types also happily accept "rubbish", or do they except?
   No, `ecdsa` and `ed25519` will raise exceptions if feed an RSA private key.
   
   ```
   self._default_pkey_mappings['dsa'].from_private_key(StringIO(private_key), password=passphrase)
   <paramiko.dsskey.DSSKey object at 0x123a611c0>
   self._default_pkey_mappings['ecdsa'].from_private_key(StringIO(private_key), password=passphrase)
   paramiko.ssh_exception.SSHException: 'utf-8' codec can't decode byte 0xb2 in position 1: invalid start byte
   self._default_pkey_mappings['ed25519'].from_private_key(StringIO(private_key), password=passphrase)
   paramiko.ssh_exception.SSHException: Invalid key
   self._default_pkey_mappings['rsa'].from_private_key(StringIO(private_key), password=passphrase)
   <paramiko.rsakey.RSAKey object at 0x123a61dc0>
   ``` 
   


-- 
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 edited a comment on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-871973743


   Seems it should be easy to fix by      `except paramiko.ssh_exception.SSHException` -> `Exception` - could you please see @ecerulm if thaf fixes the problem (even locally modifying the code) and if so - maybe you could  PR a fix for that? 
   
   UPDATE: Ah - i see it does NOT raise an exception. Then I think it can be fixed by changing key into key_candidates array and caching the key when succeeds first time. Sounds like a possible, easy solution ? WDYT?


-- 
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] ecerulm edited a comment on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ecerulm edited a comment on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872038284


   Yes, I think that could work 
   
   https://github.com/apache/airflow/blob/a2d6aa07c9de83278eef14a228eb4a02ad43205c/airflow/providers/ssh/hooks/ssh.py#L268
   
   surround that with a `try-except` and a for-loop for all the `key_candidates`   and take the first that succeeds. 
   
   Or add a `private_key_type` to extras in the connection and use that 
   
                       key = _default_pkey_mappings[private_key_type].from_private_key(StringIO(private_key), password=passphrase)
   
   


-- 
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 issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1081976299


   This ssems to be misleading message from Parmiko https://github.com/paramiko/paramiko/issues/1589  not Airflow and it seems that it might indicate you have general problems with connecting to the server. You can  chime in in the issue (and there are few similar ones as far as I can see).


-- 
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] ashb edited a comment on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872047741


   > I don't think .values() returns items in any particular order
   
   Since Py 3.6 it does, and since 3.7 it is documented as being a reliable order, and that order is the order they are defined in the source:
   
   ```
   In [4]: list(SSHHook._default_pkey_mappings.items())
   Out[4]: 
   [('dsa', paramiko.dsskey.DSSKey),
    ('ecdsa', paramiko.ecdsakey.ECDSAKey),
    ('ed25519', paramiko.ed25519key.Ed25519Key),
    ('rsa', paramiko.rsakey.RSAKey)]
   ```


-- 
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] ecerulm commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ecerulm commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872055759


   > there is not any real way around this 
   
   well as @potiuk  suggested you can catch it at https://github.com/apache/airflow/blob/a2d6aa07c9de83278eef14a228eb4a02ad43205c/airflow/providers/ssh/hooks/ssh.py#L268
   
   when you are actually trying to use the key. 
   
   Or introduce a IMHO more correct `private_key_type` in connection extras, that is the approach I use now. I did my own impl of SFTPToS3Operator and SSHHook to overcome this problem
   
   
   ```
   #
   # 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.
   import posixpath
   import stat
   from io import StringIO
   from tempfile import NamedTemporaryFile
   from typing import Optional
   from urllib.parse import urlparse
   
   import paramiko
   from airflow import AirflowException
   from airflow.models import BaseOperator
   from airflow.providers.amazon.aws.hooks.s3 import S3Hook
   from airflow.providers.ssh.hooks.ssh import SSHHook as AirflowSSHHook
   
   class SSHHook(AirflowSSHHook):
   
       def __init__(  # pylint: disable=too-many-statements
           self,
           ssh_conn_id: Optional[str] = None,
           remote_host: Optional[str] = None,
           username: Optional[str] = None,
           password: Optional[str] = None,
           key_file: Optional[str] = None,
           port: Optional[int] = None,
           timeout: int = 10,
           keepalive_interval: int = 30,
       ) -> None:
           # Use connection to override defaults
           self.private_key_type = None
           if ssh_conn_id is not None:
               conn = self.get_connection(ssh_conn_id)
               if conn.extra is not None:
                   extra_options = conn.extra_dejson
                   self.private_key_type = extra_options.get("private_key_type")
           super().__init__(ssh_conn_id, remote_host, username, password, key_file, port, timeout, keepalive_interval)
   
       def _pkey_from_private_key(self, private_key: str, passphrase: Optional[str] = None) -> paramiko.PKey:
           """
           Creates appropriate paramiko key for given private key
   
           :param private_key: string containing private key
           :return: `paramiko.PKey` appropriate for given key
           :raises AirflowException: if key cannot be read
           """
   
   
           allowed_pkey_types = self._default_pkey_mappings.values()
           if self.private_key_type:
               allowed_pkey_types = [self._default_pkey_mappings[self.private_key_type]]
           for pkey_type in allowed_pkey_types:
               try:
                   key = pkey_type.from_private_key(StringIO(private_key), password=passphrase)
                   return key
               except paramiko.ssh_exception.SSHException:
                   continue
           raise AirflowException(
               'Private key provided cannot be read by paramiko.'
               'Ensure key provided is valid for one of the following'
               'key formats: RSA, DSS, ECDSA, or Ed25519'
           )
   
   class SFTPToS3Operator(BaseOperator):
       """
       This operator enables the transferring of files from a SFTP server to
       Amazon S3.
   
       :param sftp_conn_id: The sftp connection id. The name or identifier for
           establishing a connection to the SFTP server.
       :type sftp_conn_id: str
       :param sftp_path: The sftp remote path. This is the specified file path
           for downloading the file from the SFTP server.
       :type sftp_path: str
       :param s3_conn_id: The s3 connection id. The name or identifier for
           establishing a connection to S3
       :type s3_conn_id: str
       :param s3_bucket: The targeted s3 bucket. This is the S3 bucket to where
           the file is uploaded.
       :type s3_bucket: str
       :param s3_key: The targeted s3 key. This is the specified path for
           uploading the file to S3.
       :type s3_key: str
       """
   
       template_fields = ('s3_key', 'sftp_path')
   
       def __init__(
           self,
           *,
           s3_bucket: str,
           s3_key: str,
           sftp_path: str,
           sftp_conn_id: str = 'ssh_default',
           s3_conn_id: str = 'aws_default',
           **kwargs,
       ) -> None:
           super().__init__(**kwargs)
           self.sftp_conn_id = sftp_conn_id
           self.sftp_path = sftp_path
           self.s3_bucket = s3_bucket
           self.s3_key = s3_key
           self.s3_conn_id = s3_conn_id
   
       @staticmethod
       def get_s3_key(s3_key: str) -> str:
           """This parses the correct format for S3 keys regardless of how the S3 url is passed."""
           parsed_s3_key = urlparse(s3_key)
           return parsed_s3_key.path.lstrip('/')
   
       def execute(self, context) -> None:
           self.s3_key = self.get_s3_key(self.s3_key)
           ssh_hook = SSHHook(ssh_conn_id=self.sftp_conn_id)
           s3_hook = S3Hook(self.s3_conn_id)
   
           self.sftp_client = ssh_hook.get_conn().open_sftp()
           self.sftp_list_client = ssh_hook.get_conn().open_sftp()
   
           for path,s3_key in self._walk(self.sftp_path):
               with NamedTemporaryFile("w") as f:
                   self.sftp_client.get(path, f.name)
                   self.log.info(f"Uploading SFTP file ({path}) to S3 bucket ({self.s3_bucket} and S3 key ({s3_key})")
                   s3_hook.load_file(filename=f.name, key=s3_key, bucket_name=self.s3_bucket, replace=True)
                   self.log.info(f"File uploaded to S3 {s3_key}")
   
           self.sftp_client.close()
           self.sftp_list_client.close()
   
       def _walk(self, path):
           if not stat.S_ISDIR(self.sftp_list_client.lstat(path).st_mode):
               yield (path, self.s3_key)
           else:
               yield from self._walkdir(path)
   
       def _walkdir(self, path):
           # Recurse into dir
           for x in self.sftp_list_client.listdir_iter(path):
               sftp_full_path = f"{path}/{x.filename}"
               if not stat.S_ISDIR(x.st_mode):
                   s3_rel_path = posixpath.relpath(sftp_full_path, self.sftp_path)
                   s3_full_path = posixpath.normpath(f"{self.s3_key}/{s3_rel_path}")
                   yield (sftp_full_path, s3_full_path)
               else:
                   yield from self._walkdir(sftp_full_path)
   ```


-- 
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] anto155 commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
anto155 commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1082102795


   > > As it is Airflow 1.10, initiated a discussion. Issue is not allowed for Airflow 1.10.
   > > [https://github.com/apache/airflow/discussions/22589](url)
   > 
   > BTW. Upgrade to Airflow 2. Airfow 1.10 is EOL for more than 10 months now and there were already security fixes in Airflow 2 that will never make it to 1.10 (no more updates there). So I strongly advise you to upgrade.
   > 
   > You've been warned.
   
   You are right. Meanwhile, is their any suggested solution/workaround? I am using version SSH-2.0-paramiko_2.9.1. ssh hook is being imported from  airflow.contrib.hooks.ssh_hook (airflow.providers.ssh.hooks.ssh) 
   
   File "/usr/local/airflow/.local/lib/**python3.7/site-packages/paramiko/ed25519key.py**", line 58, in init
   pkformat, data = self._read_private_key("**OPENSSH**", f)


-- 
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 issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1082134935


   > You are right. Meanwhile, is their any suggested solution/workaround? I am using version SSH-2.0-paramiko_2.9.1. ssh hook is being imported from airflow.contrib.hooks.ssh_hook (airflow.providers.ssh.hooks.ssh)
   
   From the description (if you saw it) It seems you have a problem with your key being somewhat not proper (but Paramiko telling you some lies about it ). This is not a problem with airflow. So the solution is - fix your authentication (if this is is your problem). But I do not know what problem you have. You might try your luck with Paramiko forums.


-- 
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] anto155 commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
anto155 commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1081992019


   As it is Airflow 1.10, initiated a discussion. Issue is not allowed for this version.
   
   [Using rsa key in pem format, ssh hook/paramiko seems to expect ed25519 key #22589](url)


-- 
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] anto155 commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
anto155 commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1081960548


   While using rsa key in pem format, ssh hook seem to expect ed25519 type of key as highlighted. rsa keys were working fine earlier and started having issue from last few months. Is there something recently changed?
   
   Traceback (most recent call last):
     File "/usr/local/lib/python3.7/site-packages/airflow/models/taskinstance.py", line 984, in _run_raw_task
       result = task_copy.execute(context=context)
     File "/usr/local/lib/python3.7/site-packages/airflow/contrib/operators/s3_to_sftp_operator.py", line 81, in execute
       sftp_client = ssh_hook.get_conn().open_sftp()
     File "/usr/local/lib/python3.7/site-packages/airflow/contrib/hooks/ssh_hook.py", line 194, in get_conn
       client.connect(**connect_kwargs)
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/client.py", line 446, in connect
       passphrase,
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/client.py", line 766, in _auth
       raise saved_exception
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/client.py", line 679, in _auth
       key_filename, pkey_class, passphrase
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/client.py", line 588, in _key_from_filepath
       key = klass.from_private_key_file(key_path, password)
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/pkey.py", line 249, in from_private_key_file
       key = cls(filename=filename, password=password)
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/**ed25519key.py**", line 58, in __init__
       pkformat, data = self._read_private_key("OPENSSH", f)
     File "/usr/local/airflow/.local/lib/python3.7/site-packages/paramiko/pkey.py", line 355, in _read_private_key
       "encountered {} key, expected {} key".format(keytype, tag)
   paramiko.ssh_exception.SSHException: encountered RSA key, expected OPENSSH key


-- 
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] ashb commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872047741


   > I don't think .values() returns items in any particular order
   
   Since Py 3.6 it does, and since 3.7 it is documented as being a reliable order.


-- 
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] ashb commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ashb commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872061871


   @ecerulm Do you want to PR this or shall I?


-- 
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] ecerulm commented on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
ecerulm commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-872038284


   Yes, I think that could work 
   http://gitlab.tdh.tele2.net/b2b/apache-airflow-providers-tdh/blob/1a6333dd84a4552724610b696fe1064d0a1dffbb/venv/lib/python3.8/site-packages/airflow/providers/ssh/hooks/ssh.py#L245-L246
   
   surround that with a `try-except` and a for-loop for all the `key_candidates`   and take the first that succeeds. 
   
   Or add a `private_key_type` to extras in the connection and use that 
   
                       key = _default_pkey_mappings[private_key_type].from_private_key(StringIO(private_key), password=passphrase)
   
   


-- 
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] anto155 removed a comment on issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
anto155 removed a comment on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1081992019


   As it is Airflow 1.10, initiated a discussion. Issue is not allowed for this version.
   
   [Using rsa key in pem format, ssh hook/paramiko seems to expect ed25519 key #22589](url)


-- 
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 issue #16738: SSHHook will not work if `extra.private_key` is a RSA key

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #16738:
URL: https://github.com/apache/airflow/issues/16738#issuecomment-1082135789


   BTW. It's really funny when people are told they should expect no support and then they .... expect support :).


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