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/08/01 07:32:50 UTC

[GitHub] [airflow] malthe opened a new pull request #17273: Switch to 'smbprotocol' library

malthe opened a new pull request #17273:
URL: https://github.com/apache/airflow/pull/17273


   This change switches the Samba hook implementation to the [smbprotocol](https://pypi.org/project/smbprotocol/) library.
   
   As discussed in #14054 the current implementation uses an unmaintained and outdated library.
   
   In addition, the implementation suggested here conveniently exposes a large part of the user-friendly API provided by the underlying library. For example, to use the `smbclient.getxattr` function, one would call `samba_hook.getxattr` (with connection details being automatically provided).
   
   ---
   
   closes: #14054
   


-- 
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] uranusjr commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678038843



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,231 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be
+        # perceived as a benefit), but also help work around an issue:
+        #
+        # https://github.com/jborean93/smbprotocol/issues/109.
+        smbclient.register_session(self._host, **self._conn_kwargs)
+        return self
+
+    def __exit__(self, exc_type, exc_value, traceback):
+        for host, connection in self._connection_cache.items():
+            self.log.info("Disconnecting from %s", host)
+            connection.disconnect()
+        self._connection_cache.clear()
+
+    @property
+    def _base_url(self):
+        return f"//{self._host}/{self._share}"
+
+    @wraps(smbclient.link)
+    def link(self, src, dst, follow_symlinks=True):
+        return smbclient.link(
+            self._base_url + "/" + src,

Review comment:
       Probably better to use `posixpath.join()` so users don’t need to deal with double slash issues.
   
   It may be even better to change `_base_url` to something like
   
   ```python
   def _format_url(self, endpoint):
       return f"//{posixpath.join(self._host, self._share, endpoint)}"
   ```
   
   since the value of `_base_url` is never used along but always used to generate a full path (if I didn’t miss anything).




-- 
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] malthe commented on pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#issuecomment-890590747


   @uranusjr this seems to fail for no apparant reason.


-- 
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] uranusjr commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678027923



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -31,52 +62,202 @@ class SambaHook(BaseHook):
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            LOGGER.info("Login not provided")
+
+        if not conn.password:
+            LOGGER.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,

Review comment:
       Oh OK, I misread.




-- 
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] github-actions[bot] commented on pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#issuecomment-888356926


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678042850



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,231 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be
+        # perceived as a benefit), but also help work around an issue:
+        #
+        # https://github.com/jborean93/smbprotocol/issues/109.
+        smbclient.register_session(self._host, **self._conn_kwargs)
+        return self
+
+    def __exit__(self, exc_type, exc_value, traceback):
+        for host, connection in self._connection_cache.items():
+            self.log.info("Disconnecting from %s", host)
+            connection.disconnect()
+        self._connection_cache.clear()
+
+    @property
+    def _base_url(self):
+        return f"//{self._host}/{self._share}"
+
+    @wraps(smbclient.link)
+    def link(self, src, dst, follow_symlinks=True):
+        return smbclient.link(
+            self._base_url + "/" + src,

Review comment:
       Sounds good – I did exactly 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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678021335



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -31,52 +62,202 @@ class SambaHook(BaseHook):
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            LOGGER.info("Login not provided")
+
+        if not conn.password:
+            LOGGER.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,

Review comment:
       Well, the variable is aliased locally to `connection_cache`.




-- 
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 a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678660274



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,229 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+import posixpath
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be

Review comment:
       You could see some of those in other providers - for example look at the Google provider: https://github.com/apache/airflow/blob/main/airflow/providers/google/CHANGELOG.rst - usually when there are braking changes that require more than one-line mentioning.
   
   Even in Samba, we've added a warning when we removed "apply_default" (this was a global one and it actually did NOT really apply to samba but it went there as it was "all operator" change.
   
   https://github.com/apache/airflow/blob/main/airflow/providers/samba/CHANGELOG.rst
   




-- 
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 pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#issuecomment-890590810


   seems like the errors were rather unrelated "stability" problems. Merging.


-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678032040



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,12 +16,43 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from logging import getLogger
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+from smbclient import (
+    getxattr,
+    link,
+    listdir,
+    listxattr,
+    lstat,
+    makedirs,
+    mkdir,
+    open_file,
+    readlink,
+    register_session,
+    remove,
+    removedirs,
+    removexattr,
+    rename,
+    replace,
+    rmdir,
+    scandir,
+    setxattr,
+    stat,
+    stat_volume,
+    symlink,
+    truncate,
+    unlink,
+    utime,
+    walk,
+)
 
 from airflow.hooks.base import BaseHook
 
+LOGGER = getLogger(__name__)

Review comment:
       I actually didn't know hooks were `LoggingMixin`. Nice!




-- 
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] uranusjr commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r677946939



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -31,52 +62,202 @@ class SambaHook(BaseHook):
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            LOGGER.info("Login not provided")
+
+        if not conn.password:
+            LOGGER.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,

Review comment:
       Should this be `self._connection_cache`?




-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678031771



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,12 +16,43 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from logging import getLogger
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+from smbclient import (
+    getxattr,
+    link,
+    listdir,
+    listxattr,
+    lstat,
+    makedirs,
+    mkdir,
+    open_file,
+    readlink,
+    register_session,
+    remove,
+    removedirs,
+    removexattr,
+    rename,
+    replace,
+    rmdir,
+    scandir,
+    setxattr,
+    stat,
+    stat_volume,
+    symlink,
+    truncate,
+    unlink,
+    utime,
+    walk,
+)

Review comment:
       I have updated the code now to reflect this. I think it's rare to see code that imports _from_ `os` and we do lose some import use tracking with this change, but as you say there are also some benefits.




-- 
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 merged pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #17273:
URL: https://github.com/apache/airflow/pull/17273


   


-- 
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] uranusjr commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r677940668



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,12 +16,43 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from logging import getLogger
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+from smbclient import (
+    getxattr,
+    link,
+    listdir,
+    listxattr,
+    lstat,
+    makedirs,
+    mkdir,
+    open_file,
+    readlink,
+    register_session,
+    remove,
+    removedirs,
+    removexattr,
+    rename,
+    replace,
+    rmdir,
+    scandir,
+    setxattr,
+    stat,
+    stat_volume,
+    symlink,
+    truncate,
+    unlink,
+    utime,
+    walk,
+)

Review comment:
       It’s probably better to use `import smbclient` and namespace everything under it. There are quite several function names that could be confusing an prone to coding errors (e.g. to those in `os`).




-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678644321



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,229 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+import posixpath
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be

Review comment:
       @potiuk where would those notes go? I don't see other changelog entries with such information.




-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r679695851



##########
File path: CHANGELOG.txt
##########
@@ -1,3 +1,10 @@
+Improvements
+""""""""""""
+
+- The Samba hook now uses the `smbprotocol` library, exposing a much
+  bigger set of functionality and supporting SMB2/3 protocols.

Review comment:
       @kaxil thanks – I have removed this entry again. I had actually already written a more detailed changelog entry in the provider changelog.




-- 
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] kaxil commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r679524213



##########
File path: CHANGELOG.txt
##########
@@ -1,3 +1,10 @@
+Improvements
+""""""""""""
+
+- The Samba hook now uses the `smbprotocol` library, exposing a much
+  bigger set of functionality and supporting SMB2/3 protocols.

Review comment:
       This is in wrong file -- it should be at https://github.com/apache/airflow/blob/main/airflow/providers/samba/CHANGELOG.rst
   
   




-- 
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] malthe commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678644321



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,229 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+import posixpath
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be

Review comment:
       @potiuk where would those notes go? I don't see other changelog entries with such information. I have added a simple changelog entry for now and a note in the docstring about using it as a context manager.




-- 
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 a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r678357257



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,67 +16,229 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+import posixpath
+from functools import wraps
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+import smbclient
 
 from airflow.hooks.base import BaseHook
 
 
 class SambaHook(BaseHook):
-    """Allows for interaction with an samba server."""
+    """Allows for interaction with a Samba server.
+
+    :param samba_conn_id: The connection id reference.
+    :type samba_conn_id: str
+    :param share:
+        An optional share name. If this is unset then the "schema" field of
+        the connection is used in its place.
+    :type share: str
+    """
 
     conn_name_attr = 'samba_conn_id'
     default_conn_name = 'samba_default'
     conn_type = 'samba'
     hook_name = 'Samba'
 
-    def __init__(self, samba_conn_id: str = default_conn_name) -> None:
+    def __init__(self, samba_conn_id: str = default_conn_name, share: Optional[str] = None) -> None:
         super().__init__()
-        self.conn = self.get_connection(samba_conn_id)
+        conn = self.get_connection(samba_conn_id)
+
+        if not conn.login:
+            self.log.info("Login not provided")
+
+        if not conn.password:
+            self.log.info("Password not provided")
+
+        self._host = conn.host
+        self._share = share or conn.schema
+        self._connection_cache = connection_cache = {}
+        self._conn_kwargs = {
+            "username": conn.login,
+            "password": conn.password,
+            "port": conn.port or 445,
+            "connection_cache": connection_cache,
+        }
+
+    def __enter__(self):
+        # This immediately connects to the host (which can be

Review comment:
       If I read that correctly - it means that SambaHook MUST be used as ContextManager in some cases, otherwise there might some problems with initializing some parameters during constructor? This is not a usual pattern we have in Airflow for hooks (though I think it's nice pattern for Hooks) but I think some explanation is needed at least in the docstring explaining the difference between the two and when to use it? 
   
   Also - would you mind to add a CHANGELOG.txt entry? Don't yet put the version (I will update it) but some backwards-compatibility notes are needed (how to migrate?)




-- 
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] uranusjr commented on a change in pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17273:
URL: https://github.com/apache/airflow/pull/17273#discussion_r677941361



##########
File path: airflow/providers/samba/hooks/samba.py
##########
@@ -16,12 +16,43 @@
 # specific language governing permissions and limitations
 # under the License.
 
-import os
+from functools import wraps
+from logging import getLogger
+from shutil import copyfileobj
+from typing import Optional
 
-from smbclient import SambaClient
+from smbclient import (
+    getxattr,
+    link,
+    listdir,
+    listxattr,
+    lstat,
+    makedirs,
+    mkdir,
+    open_file,
+    readlink,
+    register_session,
+    remove,
+    removedirs,
+    removexattr,
+    rename,
+    replace,
+    rmdir,
+    scandir,
+    setxattr,
+    stat,
+    stat_volume,
+    symlink,
+    truncate,
+    unlink,
+    utime,
+    walk,
+)
 
 from airflow.hooks.base import BaseHook
 
+LOGGER = getLogger(__name__)

Review comment:
       `BaseHook` is a `LoggingMixin` so we can just do `self.log.info(...)` instead of instantiating a new module-level logger. Although style around this is pretty diverse in the current code base so I’m not quite sure if `LoggingMixin` is preferred or not…




-- 
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] malthe closed pull request #17273: Switch to 'smbprotocol' library

Posted by GitBox <gi...@apache.org>.
malthe closed pull request #17273:
URL: https://github.com/apache/airflow/pull/17273


   


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