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 2022/01/12 15:07:52 UTC

[GitHub] [airflow] josh-fell commented on a change in pull request #19852: created SFTPBatchOperator which add batch function

josh-fell commented on a change in pull request #19852:
URL: https://github.com/apache/airflow/pull/19852#discussion_r783132963



##########
File path: airflow/providers/sftp/operators/sftp_batch.py
##########
@@ -0,0 +1,312 @@
+#
+# 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 contains SFTP Batch operator."""
+import ast
+import os
+import re
+from pathlib import Path
+from typing import List, Optional
+
+from paramiko.sftp_client import SFTPClient
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.sftp.operators.sftp import SFTPOperation
+from airflow.providers.sftp.utils.common import check_conn, make_intermediate_dirs
+from airflow.utils.context import Context
+
+
+class SFTPBatchOperator(BaseOperator):
+    """
+    SFTPBatchOperator for transferring files from remote host to local or vice a versa.
+    This operator uses ssh_hook to open sftp transport channel that serve as basis
+    for file transfer.
+    :param ssh_hook: predefined ssh_hook to use for remote execution.
+        Either `ssh_hook` or `ssh_conn_id` needs to be provided.
+    :type ssh_hook: airflow.providers.ssh.hooks.ssh.SSHHook
+    :param ssh_conn_id: :ref:`ssh connection id<howto/connection:ssh>`
+        from airflow Connections. `ssh_conn_id` will be ignored if `ssh_hook`
+        is provided.
+    :type ssh_conn_id: str
+    :param remote_host: remote host to connect (templated)
+        Nullable. If provided, it will replace the `remote_host` which was
+        defined in `ssh_hook` or predefined in the connection of `ssh_conn_id`.
+    :type remote_host: str
+    :param local_files_path: local files path to get or put. (templated)
+    :type local_files_path: list
+    :param local_folder: local folder path to get or put. (templated)
+    :type local_folder: str
+    :param remote_folder: remote folder path to get or put. (templated)
+    :type remote_folder: str
+    :param remote_files_path: remote folder path to get or put. (templated)
+    :type remote_files_path: list
+    :param regexp_mask: regexp mask for file match in local_folder for PUT operational
+        or match filenames in remote_folder for GET operational. (templated)
+    :type regexp_mask: str
+    :param operation: specify operation 'get' or 'put', defaults to put
+    :type operation: str
+    :param confirm: specify if the SFTP operation should be confirmed, defaults to True
+    :type confirm: bool
+    :param create_intermediate_dirs: create missing intermediate directories when
+    :type create_intermediate_dirs: bool
+    :param force: if the file already exists, it will be overwritten
+    :type force: bool
+        copying from remote to local and vice-versa. Default is False.
+    :param list_as_str: if you uses xcom or string in remote_files_path or local_files_path
+        for example your xcom return "['/tmp/tmp1/batch/file1.txt']"
+        then turn use_xcom_args = True and your will be xcom transformed to list type
+    :type list_as_str: bool
+        copying from remote to local and vice-versa. Default is False.
+    Summary, support arguments:
+        Possible options for PUT:
+            1.optional(regexp_mask:str) + local_folder:str + remote_folder:str
+            2.local_files_path:list + remote_folder:str
+        Possible options for GET:
+            1.local_folder:str + remote_folder:str + optional(regexp_mask:str)
+            2.local_folder:str + remote_files_path:list
+    Example:
+    Move all txt files
+        from local `/tmp/dir_for_local_transfer/` to remote folder `/tmp/dir_for_remote_transfer/`
+            put_dir_txt_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                regexp_mask=".*[.]txt",
+                operation=SFTPOperation.PUT,
+                create_intermediate_dirs=True
+            )
+    Move `/tmp/file1.txt` file
+        from local to remote folder `/tmp/dir_for_remote_transfer/`
+            put_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_files_path=["/tmp/file1.txt",],
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                operation=SFTPOperation.PUT,
+                create_intermediate_dirs=True
+            )
+    Move all files
+        from remote folder `/tmp/dir_for_remote_transfer/` to local folder `/tmp/dir_for_local_transfer/`
+            get_dir = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                operation=SFTPOperation.GET,
+                create_intermediate_dirs=True
+            )
+    Move `/tmp/file1.txt` file
+        from remote to local folder `/tmp/dir_for_local_transfer/`
+            get_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_files_path=["/tmp/file1.txt",],
+                operation=SFTPOperation.GET,
+                create_intermediate_dirs=True
+            )
+    """
+
+    template_fields = (
+        'remote_host',
+        'local_files_path',
+        'remote_files_path',
+        'local_folder',
+        'remote_folder',
+        'regexp_mask',
+    )
+
+    def __init__(
+        self,
+        *,
+        ssh_hook=None,
+        ssh_conn_id=None,
+        remote_host=None,
+        local_files_path: Optional[List[str]] = None,
+        remote_files_path: Optional[List[str]] = None,
+        local_folder: Optional[str] = None,
+        remote_folder: Optional[str] = None,
+        regexp_mask=None,
+        operation=SFTPOperation.PUT,
+        confirm=True,
+        create_intermediate_dirs=False,
+        force=False,
+        list_as_str=False,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.ssh_hook = ssh_hook
+        self.ssh_conn_id = ssh_conn_id
+        self.remote_host = remote_host
+        self.local_files_path = local_files_path
+        self.remote_files_path = remote_files_path
+        self.local_folder = local_folder
+        self.remote_folder = remote_folder
+        self.regexp_mask = regexp_mask
+        self.operation = operation

Review comment:
       ```suggestion
           self.operation = operation.lower()
   ```
   
   There is a little bit of a mixed bag of using `self.operation.lower()` and `self.operation` in some of the validations. When the value is not lowercase some of the validations won't work as expected. If you set the value to be lower case initially you don't need to repeat applying the function throughout.

##########
File path: airflow/providers/sftp/operators/sftp_batch.py
##########
@@ -0,0 +1,312 @@
+#
+# 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 contains SFTP Batch operator."""
+import ast
+import os
+import re
+from pathlib import Path
+from typing import List, Optional
+
+from paramiko.sftp_client import SFTPClient
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.sftp.operators.sftp import SFTPOperation
+from airflow.providers.sftp.utils.common import check_conn, make_intermediate_dirs
+from airflow.utils.context import Context
+
+
+class SFTPBatchOperator(BaseOperator):
+    """
+    SFTPBatchOperator for transferring files from remote host to local or vice a versa.
+    This operator uses ssh_hook to open sftp transport channel that serve as basis
+    for file transfer.
+    :param ssh_hook: predefined ssh_hook to use for remote execution.
+        Either `ssh_hook` or `ssh_conn_id` needs to be provided.
+    :type ssh_hook: airflow.providers.ssh.hooks.ssh.SSHHook
+    :param ssh_conn_id: :ref:`ssh connection id<howto/connection:ssh>`
+        from airflow Connections. `ssh_conn_id` will be ignored if `ssh_hook`
+        is provided.
+    :type ssh_conn_id: str
+    :param remote_host: remote host to connect (templated)
+        Nullable. If provided, it will replace the `remote_host` which was
+        defined in `ssh_hook` or predefined in the connection of `ssh_conn_id`.
+    :type remote_host: str
+    :param local_files_path: local files path to get or put. (templated)
+    :type local_files_path: list
+    :param local_folder: local folder path to get or put. (templated)
+    :type local_folder: str
+    :param remote_folder: remote folder path to get or put. (templated)
+    :type remote_folder: str
+    :param remote_files_path: remote folder path to get or put. (templated)
+    :type remote_files_path: list
+    :param regexp_mask: regexp mask for file match in local_folder for PUT operational
+        or match filenames in remote_folder for GET operational. (templated)
+    :type regexp_mask: str
+    :param operation: specify operation 'get' or 'put', defaults to put
+    :type operation: str
+    :param confirm: specify if the SFTP operation should be confirmed, defaults to True
+    :type confirm: bool
+    :param create_intermediate_dirs: create missing intermediate directories when
+    :type create_intermediate_dirs: bool
+    :param force: if the file already exists, it will be overwritten
+    :type force: bool
+        copying from remote to local and vice-versa. Default is False.
+    :param list_as_str: if you uses xcom or string in remote_files_path or local_files_path
+        for example your xcom return "['/tmp/tmp1/batch/file1.txt']"
+        then turn use_xcom_args = True and your will be xcom transformed to list type
+    :type list_as_str: bool
+        copying from remote to local and vice-versa. Default is False.

Review comment:
       There exists `render_template_as_native_obj` ([reference](https://airflow.apache.org/docs/apache-airflow/stable/concepts/operators.html#rendering-fields-as-native-python-objects)) that can be enabled at the DAG level which will turn templated values into native Python types rather than strings. I don't think you need to handle this conversion in the operator.

##########
File path: docs/apache-airflow-providers-sftp/sftp.rst
##########
@@ -0,0 +1,147 @@
+ .. 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.
+
+SFTPOperator
+==========================
+Use the :class:`~airflow.providers.sftp.operators.sftp.py` to
+transfer data between servers under sftp.
+
+Using the Operator
+------------------
+To start working with an operator, you need to register an SFTP \ SSH connection in Airflow Connections.
+Use ssh_conn_id to specify the name of the connection.

Review comment:
       I don't necessarily think you need to split the documentation especially since they are quite similar. This comment was really about making sure the documentation stated that is was about both operators rather than just `SFTPOperator`. It's very common to have one document for a grouping of operators and IMO this is a great candidate.

##########
File path: airflow/providers/sftp/operators/sftp_batch.py
##########
@@ -0,0 +1,312 @@
+#
+# 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 contains SFTP Batch operator."""
+import ast
+import os
+import re
+from pathlib import Path
+from typing import List, Optional
+
+from paramiko.sftp_client import SFTPClient
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.sftp.operators.sftp import SFTPOperation

Review comment:
       IMO the operation statuses should also come from a common location rather than an operator.

##########
File path: airflow/providers/sftp/operators/sftp_batch.py
##########
@@ -0,0 +1,312 @@
+#
+# 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 contains SFTP Batch operator."""
+import ast
+import os
+import re
+from pathlib import Path
+from typing import List, Optional
+
+from paramiko.sftp_client import SFTPClient
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.sftp.operators.sftp import SFTPOperation
+from airflow.providers.sftp.utils.common import check_conn, make_intermediate_dirs
+from airflow.utils.context import Context
+
+
+class SFTPBatchOperator(BaseOperator):
+    """
+    SFTPBatchOperator for transferring files from remote host to local or vice a versa.
+    This operator uses ssh_hook to open sftp transport channel that serve as basis
+    for file transfer.
+    :param ssh_hook: predefined ssh_hook to use for remote execution.
+        Either `ssh_hook` or `ssh_conn_id` needs to be provided.
+    :type ssh_hook: airflow.providers.ssh.hooks.ssh.SSHHook
+    :param ssh_conn_id: :ref:`ssh connection id<howto/connection:ssh>`
+        from airflow Connections. `ssh_conn_id` will be ignored if `ssh_hook`
+        is provided.
+    :type ssh_conn_id: str
+    :param remote_host: remote host to connect (templated)
+        Nullable. If provided, it will replace the `remote_host` which was
+        defined in `ssh_hook` or predefined in the connection of `ssh_conn_id`.
+    :type remote_host: str
+    :param local_files_path: local files path to get or put. (templated)
+    :type local_files_path: list
+    :param local_folder: local folder path to get or put. (templated)
+    :type local_folder: str
+    :param remote_folder: remote folder path to get or put. (templated)
+    :type remote_folder: str
+    :param remote_files_path: remote folder path to get or put. (templated)
+    :type remote_files_path: list
+    :param regexp_mask: regexp mask for file match in local_folder for PUT operational
+        or match filenames in remote_folder for GET operational. (templated)
+    :type regexp_mask: str
+    :param operation: specify operation 'get' or 'put', defaults to put
+    :type operation: str
+    :param confirm: specify if the SFTP operation should be confirmed, defaults to True
+    :type confirm: bool
+    :param create_intermediate_dirs: create missing intermediate directories when
+    :type create_intermediate_dirs: bool
+    :param force: if the file already exists, it will be overwritten
+    :type force: bool
+        copying from remote to local and vice-versa. Default is False.
+    :param list_as_str: if you uses xcom or string in remote_files_path or local_files_path
+        for example your xcom return "['/tmp/tmp1/batch/file1.txt']"
+        then turn use_xcom_args = True and your will be xcom transformed to list type
+    :type list_as_str: bool
+        copying from remote to local and vice-versa. Default is False.
+    Summary, support arguments:
+        Possible options for PUT:
+            1.optional(regexp_mask:str) + local_folder:str + remote_folder:str
+            2.local_files_path:list + remote_folder:str
+        Possible options for GET:
+            1.local_folder:str + remote_folder:str + optional(regexp_mask:str)
+            2.local_folder:str + remote_files_path:list
+    Example:
+    Move all txt files
+        from local `/tmp/dir_for_local_transfer/` to remote folder `/tmp/dir_for_remote_transfer/`
+            put_dir_txt_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                regexp_mask=".*[.]txt",
+                operation=SFTPOperation.PUT,
+                create_intermediate_dirs=True
+            )
+    Move `/tmp/file1.txt` file
+        from local to remote folder `/tmp/dir_for_remote_transfer/`
+            put_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_files_path=["/tmp/file1.txt",],
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                operation=SFTPOperation.PUT,
+                create_intermediate_dirs=True
+            )
+    Move all files
+        from remote folder `/tmp/dir_for_remote_transfer/` to local folder `/tmp/dir_for_local_transfer/`
+            get_dir = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_folder="/tmp/dir_for_remote_transfer/",
+                operation=SFTPOperation.GET,
+                create_intermediate_dirs=True
+            )
+    Move `/tmp/file1.txt` file
+        from remote to local folder `/tmp/dir_for_local_transfer/`
+            get_files = SFTPOperator(
+                task_id="put_dir_txt_files",
+                ssh_conn_id="ssh_default",
+                local_folder="/tmp/dir_for_local_transfer/",
+                remote_files_path=["/tmp/file1.txt",],
+                operation=SFTPOperation.GET,
+                create_intermediate_dirs=True
+            )
+    """
+
+    template_fields = (
+        'remote_host',
+        'local_files_path',
+        'remote_files_path',
+        'local_folder',
+        'remote_folder',
+        'regexp_mask',
+    )
+
+    def __init__(
+        self,
+        *,
+        ssh_hook=None,
+        ssh_conn_id=None,
+        remote_host=None,
+        local_files_path: Optional[List[str]] = None,
+        remote_files_path: Optional[List[str]] = None,
+        local_folder: Optional[str] = None,
+        remote_folder: Optional[str] = None,
+        regexp_mask=None,
+        operation=SFTPOperation.PUT,
+        confirm=True,
+        create_intermediate_dirs=False,
+        force=False,
+        list_as_str=False,
+        **kwargs,

Review comment:
       It would still be nice to have typing for all parameters rather than a subset. Also applicable to some of the methods. 

##########
File path: airflow/providers/sftp/operators/sftp_batch.py
##########
@@ -0,0 +1,312 @@
+#
+# 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 contains SFTP Batch operator."""
+import ast
+import os
+import re
+from pathlib import Path
+from typing import List, Optional
+
+from paramiko.sftp_client import SFTPClient
+
+from airflow.exceptions import AirflowException
+from airflow.models import BaseOperator
+from airflow.providers.sftp.operators.sftp import SFTPOperation
+from airflow.providers.sftp.utils.common import check_conn, make_intermediate_dirs
+from airflow.utils.context import Context

Review comment:
       This import needs to be under `TYPE_CHECKING` otherwise this provider will have a minimum Airflow version dependency of 2.2.3. Check out an [example](https://github.com/apache/airflow/blob/82e466d8f9b196e1efba44cc15214b9d0bd3b2d1/airflow/providers/databricks/operators/databricks.py#L28-L29) in the Databricks operator file.
   
   Then the `execute()` typing will be:
   ```python
   def execute(self, context: "Context") -> None:
   ```
   

##########
File path: docs/apache-airflow-providers-sftp/sftp_batch.rst
##########
@@ -0,0 +1,165 @@
+ .. 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.
+
+SFTPBatchOperator
+==========================
+Use the :class:`~airflow.providers.sftp.operators.sftp_batch.py` to
+transfer data between servers under sftp.
+
+Using the Operator
+------------------
+To start working with an operator, you need to register an SFTP \ SSH connection in Airflow Connections.
+Use ssh_conn_id to specify the name of the connection.
+
+You can use the operator for the following tasks:
+
+1. Send all files from local directory to remote server
+
+.. code-block:: python
+
+    put_dir_files = SFTPBatchOperator(
+        task_id="put_dir_files",
+        ssh_conn_id="ssh_default",
+        local_folder="/tmp/local_folder/",
+        remote_folder="/tmp/dir_for_remote_transfer/",
+        operation=SFTPOperation.PUT,
+        create_intermediate_dirs=True,
+    )
+
+
+2. Send specific files from local directory to remote server
+
+.. code-block:: python
+
+    put_dir_files = SFTPBatchOperator(
+        task_id="put_dir_files",
+        ssh_conn_id="ssh_default",
+        local_files_path=[
+            "/tmp/local_folder/file1.txt",
+        ],
+        remote_folder="/tmp/dir_for_remote_transfer/",
+        operation=SFTPOperation.PUT,
+        create_intermediate_dirs=True,
+    )
+
+
+3. Send all files from the local directory that match the specified pattern to the remote server
+
+.. code-block:: python
+
+    put_dir_txt_files = SFTPBatchOperator(
+        task_id="put_dir_txt_files",
+        ssh_conn_id="ssh_default",
+        local_folder="/tmp/local_folder/",
+        remote_folder="/tmp/dir_for_remote_transfer/",
+        regexp_mask=r".*\.txt",
+        operation=SFTPOperation.PUT,
+        create_intermediate_dirs=True,
+    )
+
+
+4. Get specific list of files from the remote server to the local folder
+
+.. code-block:: python
+
+    put_dir_txt_files = SFTPBatchOperator(
+        task_id="put_dir_txt_files",
+        ssh_conn_id="ssh_default",
+        local_folder="/tmp/local_folder/",
+        remote_files_path=[
+            "/tmp/dir_for_remote_transfer/file1.txt",
+            "/tmp/dir_for_remote_transfer/file2.txt",
+        ],
+        operation=SFTPOperation.GET,
+        create_intermediate_dirs=True,
+    )
+
+
+5. Get all files from the remote server to the local folder
+
+.. code-block:: python
+
+    put_dir_txt_files = SFTPBatchOperator(
+        task_id="put_dir_txt_files",
+        ssh_conn_id="ssh_default",
+        local_folder="/tmp/local_folder/",
+        remote_folder="/tmp/dir_for_remote_transfer/",
+        operation=SFTPOperation.GET,
+        create_intermediate_dirs=True,
+    )
+
+
+6. Get all files from the remote server that match the specified pattern to the local folder with overwrite files
+
+.. code-block:: python
+
+    put_dir_txt_files = SFTPBatchOperator(
+        task_id="put_dir_txt_files",
+        ssh_conn_id="ssh_default",
+        local_folder="/tmp/local_folder/",
+        remote_folder="/tmp/dir_for_remote_transfer/",
+        regexp_mask=r".*\.txt",
+        operation=SFTPOperation.GET,
+        create_intermediate_dirs=True,
+        force=True,
+    )
+
+
+7. You can use xcom for local_files_path or remote_files_path arguments,
+    when you do `xcom_pull` in arguments, then your xcom will be transformed to string type,
+    and you can use `list_as_str` argument for this case
+
+.. code-block:: python
+
+    def push_xcom(**context):
+        ti: TaskInstance = context["task_instance"]
+        # save as list
+        ti.xcom_push(
+            key="file_names", value=["/some_path/with_complex_logic/some_file.txt"]
+        )
+
+
+    PythonOperator(
+        task_id="push_xcom",
+        python_callable=push_xcom,
+        provide_context=True,

Review comment:
       This is a deprecated parameter.
   ```suggestion
   ```




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