You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "pankajastro (via GitHub)" <gi...@apache.org> on 2023/08/30 01:39:08 UTC

[GitHub] [airflow] pankajastro opened a new pull request, #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

pankajastro opened a new pull request, #33904:
URL: https://github.com/apache/airflow/pull/33904

   TODO: Add why?
   
   <!--
    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.
    -->
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   
   
   <!-- Please keep an empty line above the dashes. -->
   ---
   **^ Add meaningful description above**
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
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] pankajastro commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1312937312


##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -31,20 +30,31 @@ class AzureFileShareHook(BaseHook):
     Interacts with Azure FileShare Storage.
 
     :param azure_fileshare_conn_id: Reference to the
-        :ref:`Azure Container Volume connection id<howto/connection:azure_fileshare>`
-        of an Azure account of which container volumes should be used.
-
+        :ref:`Azure FileShare connection id<howto/connection:azure_fileshare>`
+        of an Azure account of which file share should be used.
     """
 
     conn_name_attr = "azure_fileshare_conn_id"
     default_conn_name = "azure_fileshare_default"
     conn_type = "azure_fileshare"
     hook_name = "Azure FileShare"
 
-    def __init__(self, azure_fileshare_conn_id: str = "azure_fileshare_default") -> None:
+    def __init__(
+        self,
+        share_name: str | None = None,
+        file_path: str | None = None,
+        directory_path: str | None = None,
+        azure_fileshare_conn_id: str = "azure_fileshare_default",
+    ) -> None:
         super().__init__()
-        self.conn_id = azure_fileshare_conn_id
-        self._conn = None
+        self._conn_id = azure_fileshare_conn_id
+        self.share_name = share_name
+        self.file_path = file_path
+        self.directory_path = directory_path
+        self._account_url: str | None = None
+        self._connection_string = None
+        self._account_access_key = None
+        self._sas_token = None

Review Comment:
   Added, PTAL.
   
   



-- 
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] Lee-W commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "Lee-W (via GitHub)" <gi...@apache.org>.
Lee-W commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1313135108


##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:

Review Comment:
   If that's the case, should we add a message to avoid confusion? calling `get_conn` and return nothing might be confusing



-- 
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] phanikumv commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "phanikumv (via GitHub)" <gi...@apache.org>.
phanikumv commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1312685973


##########
airflow/providers/microsoft/azure/CHANGELOG.rst:
##########
@@ -27,6 +27,31 @@
 Changelog
 ---------
 
+7.0.0
+.....
+
+Breaking changes
+~~~~~~~~~~~~~~~~
+
+.. warning::
+  In this version of the provider, we have changed AzureFileShareHook to use azure-storage-file-share API instead of azure-storage-file,

Review Comment:
   ```suggestion
     In this version of the provider, we have changed AzureFileShareHook to use azure-storage-file-share library instead of azure-storage-file,
   ```



-- 
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] pankajastro commented on pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on PR #33904:
URL: https://github.com/apache/airflow/pull/33904#issuecomment-1698423222

   Hi @potiuk / @eladkal I'm trying to update `AzureFileShareHook` to use the newly updated Azure SDK but interfaces on the Azure SDK side have changed a lot which makes it difficult to deprecate the existing hook API and same time update it to use the new Azure SDK.  I'm thinking either
   1. Add warring about Deprecation and release it and then the next release replaces the deprecated one with this PR  
   2. Or, do a breaking release 
   
   I would like hear your thoughts on this


-- 
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] Lee-W commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "Lee-W (via GitHub)" <gi...@apache.org>.
Lee-W commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1313135108


##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:

Review Comment:
   If that's the case, should we add a log to avoid confusion?



-- 
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] alexbegg commented on pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "alexbegg (via GitHub)" <gi...@apache.org>.
alexbegg commented on PR #33904:
URL: https://github.com/apache/airflow/pull/33904#issuecomment-1701487432

   I added an issue for this recently. Can you add to the description that this closes https://github.com/apache/airflow/issues/33850
   
   Thanks


-- 
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] pankajastro commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1312937552


##########
airflow/providers/microsoft/azure/CHANGELOG.rst:
##########
@@ -27,6 +27,31 @@
 Changelog
 ---------
 
+7.0.0
+.....
+
+Breaking changes
+~~~~~~~~~~~~~~~~
+
+.. warning::
+  In this version of the provider, we have changed AzureFileShareHook to use azure-storage-file-share API instead of azure-storage-file,
+  removed deprecated extra__azure_fileshare__ prefix from connection extras and removed protocol param from connection extras
+
+* get_conn from AzureFileShareHook return None instead FileService
+* Remove protocol param from Azure fileshare connection extras
+* Remove deprecated extra__azure_fileshare__ prefix from Azure fileshare connection extras
+* Remove share_name, directory_name param from AzureFileShareHook.check_for_directory in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name param from AzureFileShareHook.check_for_file method in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name param from AzureFileShareHook.list_directories_and_files
+* Remove share_name, directory_name param from AzureFileShareHook.list_files in favor of AzureFileShareHook share_name and directory_path param
+* AzureFileShareHook method create_share accept kwargs from ShareServiceClient.create_share instead of FileService.create_share
+* AzureFileShareHook method delete_share accept kwargs from ShareServiceClient.delete_share instead of FileService.delete_share
+* Remove share_name, directory_name param from AzureFileShareHook.create_directory in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.get_file in favor of AzureFileShareHook share_name and file_path
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.get_file_to_stream in favor of AzureFileShareHook share_name and file_path
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.load_file in favor of AzureFileShareHook share_name and file_path
+* Remove AzureFileShareHook.load_string, AzureFileShareHook.load_stream in favor of AzureFileShareHook.load_data

Review Comment:
   I have updated it a bit, PTAL



-- 
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] pankajastro commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "pankajastro (via GitHub)" <gi...@apache.org>.
pankajastro commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1312937141


##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:

Review Comment:
   azure-storage-file-share exposes multiple clients, so I have added some property (share_service_client, share_directory_client, share_file_client) to access them instead of returning just one client from this method. 
   
   



-- 
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] eladkal commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1313146345


##########
airflow/providers/microsoft/azure/CHANGELOG.rst:
##########
@@ -27,6 +27,29 @@
 Changelog
 ---------
 
+7.0.0

Review Comment:
   If this is a breaking change please add 7.0.0 entry to provider.yaml



-- 
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] eladkal merged pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal merged PR #33904:
URL: https://github.com/apache/airflow/pull/33904


-- 
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] eladkal commented on pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "eladkal (via GitHub)" <gi...@apache.org>.
eladkal commented on PR #33904:
URL: https://github.com/apache/airflow/pull/33904#issuecomment-1698501110

   We can do a major release. I think this case justify it.
   Add entry in provider.yaml and make sure to add detailed information about what changed in the top of 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] Lee-W commented on a diff in pull request #33904: Update Azure fileshare hook to use azure-storage-file-share instead of azure-storage-file

Posted by "Lee-W (via GitHub)" <gi...@apache.org>.
Lee-W commented on code in PR #33904:
URL: https://github.com/apache/airflow/pull/33904#discussion_r1311374537


##########
airflow/providers/microsoft/azure/CHANGELOG.rst:
##########
@@ -27,6 +27,31 @@
 Changelog
 ---------
 
+7.0.0
+.....
+
+Breaking changes
+~~~~~~~~~~~~~~~~
+
+.. warning::
+  In this version of the provider, we have changed AzureFileShareHook to use azure-storage-file-share API instead of azure-storage-file,
+  removed deprecated extra__azure_fileshare__ prefix from connection extras and removed protocol param from connection extras
+
+* get_conn from AzureFileShareHook return None instead FileService
+* Remove protocol param from Azure fileshare connection extras
+* Remove deprecated extra__azure_fileshare__ prefix from Azure fileshare connection extras
+* Remove share_name, directory_name param from AzureFileShareHook.check_for_directory in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name param from AzureFileShareHook.check_for_file method in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name param from AzureFileShareHook.list_directories_and_files
+* Remove share_name, directory_name param from AzureFileShareHook.list_files in favor of AzureFileShareHook share_name and directory_path param
+* AzureFileShareHook method create_share accept kwargs from ShareServiceClient.create_share instead of FileService.create_share
+* AzureFileShareHook method delete_share accept kwargs from ShareServiceClient.delete_share instead of FileService.delete_share
+* Remove share_name, directory_name param from AzureFileShareHook.create_directory in favor of AzureFileShareHook share_name and directory_path param
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.get_file in favor of AzureFileShareHook share_name and file_path
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.get_file_to_stream in favor of AzureFileShareHook share_name and file_path
+* Remove share_name, directory_name, file_name param from AzureFileShareHook.load_file in favor of AzureFileShareHook share_name and file_path
+* Remove AzureFileShareHook.load_string, AzureFileShareHook.load_stream in favor of AzureFileShareHook.load_data

Review Comment:
   ```suggestion
   * Remove protocol param from Azure fileshare connection extras
   * Remove deprecated extra__azure_fileshare__ prefix from Azure fileshare connection extras
   * Remove share_name, directory_name param from the following in favor of AzureFileShareHook share_name and directory_path param
       * AzureFileShareHook.check_for_directory
       * AzureFileShareHook.check_for_file
       * AzureFileShareHook.list_files
       * AzureFileShareHook.create_directory
       * AzureFileShareHook.get_file
       * AzureFileShareHook.get_file_to_stream
       * AzureFileShareHook.load_file
   * Remove share_name, directory_name param from AzureFileShareHook.list_directories_and_files
   * AzureFileShareHook method create_share accept kwargs from ShareServiceClient.create_share instead of FileService.create_share
   * AzureFileShareHook method delete_share accept kwargs from ShareServiceClient.delete_share instead of FileService.delete_share
   * Remove AzureFileShareHook.load_string, AzureFileShareHook.load_stream in favor of AzureFileShareHook.load_data
   ```
   Should we consolidate similar changes into one nested list?



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:
+        conn = self.get_connection(self._conn_id)
         extras = conn.extra_dejson
-        service_options = {}
-        for key, value in extras.items():
-            if value == "":
-                continue
-            if not key.startswith("extra__"):
-                service_options[key] = value
-                check_for_conflict(key)
-            elif key.startswith(backcompat_prefix):
-                short_name = key[len(backcompat_prefix) :]
-                warnings.warn(
-                    f"`{key}` is deprecated in azure connection extra please use `{short_name}` instead",
-                    AirflowProviderDeprecationWarning,
-                    stacklevel=2,
-                )
-                if short_name not in service_options:  # prefer values provided with short name
-                    service_options[short_name] = value
-            else:
-                warnings.warn(f"Extra param `{key}` not recognized; ignoring.")
-        self._conn = FileService(account_name=conn.login, account_key=conn.password, **service_options)
-        return self._conn
-
-    def check_for_directory(self, share_name: str, directory_name: str, **kwargs) -> bool:
-        """
-        Check if a directory exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, **kwargs)
-
-    def check_for_file(self, share_name: str, directory_name: str, file_name: str, **kwargs) -> bool:
-        """
-        Check if a file exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param file_name: Name of the file.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, file_name, **kwargs)
-
-    def list_directories_and_files(
-        self, share_name: str, directory_name: str | None = None, **kwargs
-    ) -> list:
-        """
-        Return the list of directories and files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files and directories
-        """
-        return self.get_conn().list_directories_and_files(share_name, directory_name, **kwargs)
+        self._connection_string = extras.get("connection_string")
+        if conn.login:
+            self._account_url = self._parse_account_url(conn.login)
+        self._sas_token = extras.get("sas_token")
+        self._account_access_key = conn.password
 
-    def list_files(self, share_name: str, directory_name: str | None = None, **kwargs) -> list[str]:
-        """
-        Return the list of files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files
-        """
-        return [
-            obj.name
-            for obj in self.list_directories_and_files(share_name, directory_name, **kwargs)
-            if isinstance(obj, File)
-        ]
+    @staticmethod
+    def _parse_account_url(account_url: str) -> str:
+        if not account_url.lower().startswith("https"):
+            return f"https://{account_url}.file.core.windows.net"
+        return account_url
+
+    @property
+    def share_service_client(self):
+
+        self.get_conn()
+        if self._connection_string:
+            return ShareServiceClient.from_connection_string(
+                conn_str=self._connection_string,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareServiceClient(account_url=self._account_url, credential=self._account_access_key)
+        elif self._account_url and self._sas_token:
+            return ShareServiceClient(account_url=self._account_url, credential=self._sas_token)

Review Comment:
   What about 
   ```suggestion
           elif self._account_url and (self._account_access_key or self._sas_token):
               credential = self._account_access_key or self._sas_token
               return ShareServiceClient(account_url=self._account_url, credential=credential)
   ```



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:
+        conn = self.get_connection(self._conn_id)
         extras = conn.extra_dejson
-        service_options = {}
-        for key, value in extras.items():
-            if value == "":
-                continue
-            if not key.startswith("extra__"):
-                service_options[key] = value
-                check_for_conflict(key)
-            elif key.startswith(backcompat_prefix):
-                short_name = key[len(backcompat_prefix) :]
-                warnings.warn(
-                    f"`{key}` is deprecated in azure connection extra please use `{short_name}` instead",
-                    AirflowProviderDeprecationWarning,
-                    stacklevel=2,
-                )
-                if short_name not in service_options:  # prefer values provided with short name
-                    service_options[short_name] = value
-            else:
-                warnings.warn(f"Extra param `{key}` not recognized; ignoring.")
-        self._conn = FileService(account_name=conn.login, account_key=conn.password, **service_options)
-        return self._conn
-
-    def check_for_directory(self, share_name: str, directory_name: str, **kwargs) -> bool:
-        """
-        Check if a directory exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, **kwargs)
-
-    def check_for_file(self, share_name: str, directory_name: str, file_name: str, **kwargs) -> bool:
-        """
-        Check if a file exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param file_name: Name of the file.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, file_name, **kwargs)
-
-    def list_directories_and_files(
-        self, share_name: str, directory_name: str | None = None, **kwargs
-    ) -> list:
-        """
-        Return the list of directories and files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files and directories
-        """
-        return self.get_conn().list_directories_and_files(share_name, directory_name, **kwargs)
+        self._connection_string = extras.get("connection_string")
+        if conn.login:
+            self._account_url = self._parse_account_url(conn.login)
+        self._sas_token = extras.get("sas_token")
+        self._account_access_key = conn.password
 
-    def list_files(self, share_name: str, directory_name: str | None = None, **kwargs) -> list[str]:
-        """
-        Return the list of files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files
-        """
-        return [
-            obj.name
-            for obj in self.list_directories_and_files(share_name, directory_name, **kwargs)
-            if isinstance(obj, File)
-        ]
+    @staticmethod
+    def _parse_account_url(account_url: str) -> str:
+        if not account_url.lower().startswith("https"):
+            return f"https://{account_url}.file.core.windows.net"
+        return account_url
+
+    @property
+    def share_service_client(self):
+
+        self.get_conn()
+        if self._connection_string:
+            return ShareServiceClient.from_connection_string(
+                conn_str=self._connection_string,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareServiceClient(account_url=self._account_url, credential=self._account_access_key)
+        elif self._account_url and self._sas_token:
+            return ShareServiceClient(account_url=self._account_url, credential=self._sas_token)
+        else:
+            return ShareServiceClient(
+                account_url=self._account_url, credential=DefaultAzureCredential(), token_intent="backup"
+            )
+
+    @property
+    def share_directory_client(self):
+
+        if self._connection_string:
+            return ShareDirectoryClient.from_connection_string(
+                conn_str=self._connection_string,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareDirectoryClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+                credential=self._account_access_key,
+            )
+        elif self._account_url and self._sas_token:
+            return ShareDirectoryClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+                credential=self._sas_token,
+            )
+        else:
+            return ShareDirectoryClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+                credential=DefaultAzureCredential(),
+                token_intent="backup",
+            )
+
+    @property
+    def share_file_client(self):
+        if self._connection_string:
+            return ShareFileClient.from_connection_string(
+                conn_str=self._connection_string,
+                share_name=self.share_name,
+                file_path=self.file_path,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareFileClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                file_path=self.file_path,
+                credential=self._account_access_key,
+            )
+        elif self._account_url and self._sas_token:
+            return ShareFileClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                file_path=self.file_path,
+                credential=self._sas_token,
+            )
+        else:

Review Comment:
   What about 
   
   ```suggestion
           elif self._account_url and (self._account_access_key or self._sas_token):
               credential = self._account_access_key or self._sas_token
               return ShareFileClient(
                   account_url=self._account_url,
                   share_name=self.share_name,
                   file_path=self.file_path,
                   credential=credential,
               )
   ```



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:
+        conn = self.get_connection(self._conn_id)
         extras = conn.extra_dejson
-        service_options = {}
-        for key, value in extras.items():
-            if value == "":
-                continue
-            if not key.startswith("extra__"):
-                service_options[key] = value
-                check_for_conflict(key)
-            elif key.startswith(backcompat_prefix):
-                short_name = key[len(backcompat_prefix) :]
-                warnings.warn(
-                    f"`{key}` is deprecated in azure connection extra please use `{short_name}` instead",
-                    AirflowProviderDeprecationWarning,
-                    stacklevel=2,
-                )
-                if short_name not in service_options:  # prefer values provided with short name
-                    service_options[short_name] = value
-            else:
-                warnings.warn(f"Extra param `{key}` not recognized; ignoring.")
-        self._conn = FileService(account_name=conn.login, account_key=conn.password, **service_options)
-        return self._conn
-
-    def check_for_directory(self, share_name: str, directory_name: str, **kwargs) -> bool:
-        """
-        Check if a directory exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, **kwargs)
-
-    def check_for_file(self, share_name: str, directory_name: str, file_name: str, **kwargs) -> bool:
-        """
-        Check if a file exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param file_name: Name of the file.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, file_name, **kwargs)
-
-    def list_directories_and_files(
-        self, share_name: str, directory_name: str | None = None, **kwargs
-    ) -> list:
-        """
-        Return the list of directories and files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files and directories
-        """
-        return self.get_conn().list_directories_and_files(share_name, directory_name, **kwargs)
+        self._connection_string = extras.get("connection_string")
+        if conn.login:
+            self._account_url = self._parse_account_url(conn.login)
+        self._sas_token = extras.get("sas_token")
+        self._account_access_key = conn.password
 
-    def list_files(self, share_name: str, directory_name: str | None = None, **kwargs) -> list[str]:
-        """
-        Return the list of files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files
-        """
-        return [
-            obj.name
-            for obj in self.list_directories_and_files(share_name, directory_name, **kwargs)
-            if isinstance(obj, File)
-        ]
+    @staticmethod
+    def _parse_account_url(account_url: str) -> str:
+        if not account_url.lower().startswith("https"):
+            return f"https://{account_url}.file.core.windows.net"
+        return account_url
+
+    @property
+    def share_service_client(self):
+

Review Comment:
   ```suggestion
   ```



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:

Review Comment:
   May I know why do we want to return None for this method? I thought we'd get a file service client



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:
+        conn = self.get_connection(self._conn_id)
         extras = conn.extra_dejson
-        service_options = {}
-        for key, value in extras.items():
-            if value == "":
-                continue
-            if not key.startswith("extra__"):
-                service_options[key] = value
-                check_for_conflict(key)
-            elif key.startswith(backcompat_prefix):
-                short_name = key[len(backcompat_prefix) :]
-                warnings.warn(
-                    f"`{key}` is deprecated in azure connection extra please use `{short_name}` instead",
-                    AirflowProviderDeprecationWarning,
-                    stacklevel=2,
-                )
-                if short_name not in service_options:  # prefer values provided with short name
-                    service_options[short_name] = value
-            else:
-                warnings.warn(f"Extra param `{key}` not recognized; ignoring.")
-        self._conn = FileService(account_name=conn.login, account_key=conn.password, **service_options)
-        return self._conn
-
-    def check_for_directory(self, share_name: str, directory_name: str, **kwargs) -> bool:
-        """
-        Check if a directory exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, **kwargs)
-
-    def check_for_file(self, share_name: str, directory_name: str, file_name: str, **kwargs) -> bool:
-        """
-        Check if a file exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param file_name: Name of the file.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, file_name, **kwargs)
-
-    def list_directories_and_files(
-        self, share_name: str, directory_name: str | None = None, **kwargs
-    ) -> list:
-        """
-        Return the list of directories and files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files and directories
-        """
-        return self.get_conn().list_directories_and_files(share_name, directory_name, **kwargs)
+        self._connection_string = extras.get("connection_string")
+        if conn.login:
+            self._account_url = self._parse_account_url(conn.login)
+        self._sas_token = extras.get("sas_token")
+        self._account_access_key = conn.password
 
-    def list_files(self, share_name: str, directory_name: str | None = None, **kwargs) -> list[str]:
-        """
-        Return the list of files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files
-        """
-        return [
-            obj.name
-            for obj in self.list_directories_and_files(share_name, directory_name, **kwargs)
-            if isinstance(obj, File)
-        ]
+    @staticmethod
+    def _parse_account_url(account_url: str) -> str:
+        if not account_url.lower().startswith("https"):
+            return f"https://{account_url}.file.core.windows.net"
+        return account_url
+
+    @property
+    def share_service_client(self):
+
+        self.get_conn()
+        if self._connection_string:
+            return ShareServiceClient.from_connection_string(
+                conn_str=self._connection_string,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareServiceClient(account_url=self._account_url, credential=self._account_access_key)
+        elif self._account_url and self._sas_token:
+            return ShareServiceClient(account_url=self._account_url, credential=self._sas_token)
+        else:
+            return ShareServiceClient(
+                account_url=self._account_url, credential=DefaultAzureCredential(), token_intent="backup"
+            )
+
+    @property
+    def share_directory_client(self):
+
+        if self._connection_string:
+            return ShareDirectoryClient.from_connection_string(
+                conn_str=self._connection_string,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareDirectoryClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+                credential=self._account_access_key,
+            )
+        elif self._account_url and self._sas_token:
+            return ShareDirectoryClient(
+                account_url=self._account_url,
+                share_name=self.share_name,
+                directory_path=self.directory_path,
+                credential=self._sas_token,
+            )

Review Comment:
   What about 
   
   ```suggestion
           elif self._account_url and (self._account_access_key or self._sas_token):
               credential = self._account_access_key or self._sas_token
               return ShareDirectoryClient(
                   account_url=self._account_url,
                   share_name=self.share_name,
                   directory_path=self.directory_path,
                   credential=credential,
               )
   ```



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -73,226 +80,195 @@ def get_ui_field_behaviour() -> dict[str, Any]:
                 "password": "Blob Storage Key (optional)",
             },
             "placeholders": {
-                "login": "account name",
+                "login": "account name or account url",
                 "password": "secret",
                 "sas_token": "account url or token (optional)",
                 "connection_string": "account url or token (optional)",
-                "protocol": "account url or token (optional)",
             },
         }
 
-    def get_conn(self) -> FileService:
-        """Return the FileService object."""
-
-        def check_for_conflict(key):
-            backcompat_key = f"{backcompat_prefix}{key}"
-            if backcompat_key in extras:
-                warnings.warn(
-                    f"Conflicting params `{key}` and `{backcompat_key}` found in extras for conn "
-                    f"{self.conn_id}. Using value for `{key}`.  Please ensure this is the correct value "
-                    f"and remove the backcompat key `{backcompat_key}`."
-                )
-
-        backcompat_prefix = "extra__azure_fileshare__"
-        if self._conn:
-            return self._conn
-        conn = self.get_connection(self.conn_id)
+    def get_conn(self) -> None:
+        conn = self.get_connection(self._conn_id)
         extras = conn.extra_dejson
-        service_options = {}
-        for key, value in extras.items():
-            if value == "":
-                continue
-            if not key.startswith("extra__"):
-                service_options[key] = value
-                check_for_conflict(key)
-            elif key.startswith(backcompat_prefix):
-                short_name = key[len(backcompat_prefix) :]
-                warnings.warn(
-                    f"`{key}` is deprecated in azure connection extra please use `{short_name}` instead",
-                    AirflowProviderDeprecationWarning,
-                    stacklevel=2,
-                )
-                if short_name not in service_options:  # prefer values provided with short name
-                    service_options[short_name] = value
-            else:
-                warnings.warn(f"Extra param `{key}` not recognized; ignoring.")
-        self._conn = FileService(account_name=conn.login, account_key=conn.password, **service_options)
-        return self._conn
-
-    def check_for_directory(self, share_name: str, directory_name: str, **kwargs) -> bool:
-        """
-        Check if a directory exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, **kwargs)
-
-    def check_for_file(self, share_name: str, directory_name: str, file_name: str, **kwargs) -> bool:
-        """
-        Check if a file exists on Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param file_name: Name of the file.
-        :param kwargs: Optional keyword arguments that
-            `FileService.exists()` takes.
-        :return: True if the file exists, False otherwise.
-        """
-        return self.get_conn().exists(share_name, directory_name, file_name, **kwargs)
-
-    def list_directories_and_files(
-        self, share_name: str, directory_name: str | None = None, **kwargs
-    ) -> list:
-        """
-        Return the list of directories and files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files and directories
-        """
-        return self.get_conn().list_directories_and_files(share_name, directory_name, **kwargs)
+        self._connection_string = extras.get("connection_string")
+        if conn.login:
+            self._account_url = self._parse_account_url(conn.login)
+        self._sas_token = extras.get("sas_token")
+        self._account_access_key = conn.password
 
-    def list_files(self, share_name: str, directory_name: str | None = None, **kwargs) -> list[str]:
-        """
-        Return the list of files stored on a Azure File Share.
-
-        :param share_name: Name of the share.
-        :param directory_name: Name of the directory.
-        :param kwargs: Optional keyword arguments that
-            `FileService.list_directories_and_files()` takes.
-        :return: A list of files
-        """
-        return [
-            obj.name
-            for obj in self.list_directories_and_files(share_name, directory_name, **kwargs)
-            if isinstance(obj, File)
-        ]
+    @staticmethod
+    def _parse_account_url(account_url: str) -> str:
+        if not account_url.lower().startswith("https"):
+            return f"https://{account_url}.file.core.windows.net"
+        return account_url
+
+    @property
+    def share_service_client(self):
+
+        self.get_conn()
+        if self._connection_string:
+            return ShareServiceClient.from_connection_string(
+                conn_str=self._connection_string,
+            )
+        elif self._account_url and self._account_access_key:
+            return ShareServiceClient(account_url=self._account_url, credential=self._account_access_key)
+        elif self._account_url and self._sas_token:
+            return ShareServiceClient(account_url=self._account_url, credential=self._sas_token)
+        else:
+            return ShareServiceClient(
+                account_url=self._account_url, credential=DefaultAzureCredential(), token_intent="backup"
+            )
+
+    @property
+    def share_directory_client(self):
+

Review Comment:
   ```suggestion
   ```



##########
airflow/providers/microsoft/azure/hooks/fileshare.py:
##########
@@ -31,20 +30,31 @@ class AzureFileShareHook(BaseHook):
     Interacts with Azure FileShare Storage.
 
     :param azure_fileshare_conn_id: Reference to the
-        :ref:`Azure Container Volume connection id<howto/connection:azure_fileshare>`
-        of an Azure account of which container volumes should be used.
-
+        :ref:`Azure FileShare connection id<howto/connection:azure_fileshare>`
+        of an Azure account of which file share should be used.
     """
 
     conn_name_attr = "azure_fileshare_conn_id"
     default_conn_name = "azure_fileshare_default"
     conn_type = "azure_fileshare"
     hook_name = "Azure FileShare"
 
-    def __init__(self, azure_fileshare_conn_id: str = "azure_fileshare_default") -> None:
+    def __init__(
+        self,
+        share_name: str | None = None,
+        file_path: str | None = None,
+        directory_path: str | None = None,
+        azure_fileshare_conn_id: str = "azure_fileshare_default",
+    ) -> None:
         super().__init__()
-        self.conn_id = azure_fileshare_conn_id
-        self._conn = None
+        self._conn_id = azure_fileshare_conn_id
+        self.share_name = share_name
+        self.file_path = file_path
+        self.directory_path = directory_path
+        self._account_url: str | None = None
+        self._connection_string = None
+        self._account_access_key = None
+        self._sas_token = None

Review Comment:
   Should we annotate these values?



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