You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/12/09 16:44:15 UTC

[GitHub] [airflow] bharanidharan14 opened a new pull request, #28262: [WIP]: Hook for managing directories and files in Azure Data Lake Storage Gen2

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

   - created hook for supporting ADLS gen2 


-- 
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] luanmorenomaciel commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
luanmorenomaciel commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1053371248


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   We should be fine to inherit from BaseHook



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049291368


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):
+
+    conn_name_attr = "adls_v2_conn_id"
+    default_conn_name = "adls_v2_default"
+    conn_type = "adls_v2"
+    hook_name = "Azure Date Lake Storage"
+
+    @staticmethod
+    def get_connection_form_widgets() -> dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3PasswordFieldWidget, BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import PasswordField, StringField
+
+        return {
+            "extra__adls_v2__connection_string": PasswordField(
+                lazy_gettext("Blob Storage Connection String (optional)"), widget=BS3PasswordFieldWidget()
+            ),
+            "extra__adls_v2__tenant_id": StringField(
+                lazy_gettext("Tenant Id (Active Directory Auth)"), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ["schema", "port"],
+            "relabeling": {
+                "login": "Blob Storage Login (optional)",
+                "password": "Blob Storage Key (optional)",
+                "host": "Account Name (Active Directory Auth)",
+            },
+            "placeholders": {
+                "login": "account name",
+                "password": "secret",
+                "host": "account url",
+                "extra__adls_v2__connection_string": "connection string auth",
+                "extra__adls_v2__tenant_id": "tenant",
+            },
+        }
+
+    def __init__(self, adls_v2_conn_id: str = default_conn_name, public_read: bool = False) -> None:
+        super().__init__()
+        self.conn_id = adls_v2_conn_id
+        self.public_read = public_read
+        self.service_client = self.get_conn()
+
+    def get_conn(self) -> DataLakeServiceClient:
+        """Return the DataLakeServiceClient object."""
+        conn = self.get_connection(self.conn_id)
+        extra = conn.extra_dejson or {}
+
+        connection_string = extra.pop(
+            "connection_string", extra.pop("extra__adls_v2__connection_string", None)
+        )
+        if connection_string:
+            # connection_string auth takes priority
+            return DataLakeServiceClient.from_connection_string(connection_string, **extra)
+
+        tenant = extra.pop("tenant_id", extra.pop("extra__adls_v2__tenant_id", None))
+        if tenant:
+            # use Active Directory auth
+            app_id = conn.login
+            app_secret = conn.password
+            token_credential = ClientSecretCredential(tenant, app_id, app_secret)
+            return DataLakeServiceClient(
+                account_url=f"https://{conn.login}.dfs.core.windows.net", credential=token_credential, **extra
+            )
+        credential = conn.password

Review Comment:
   @tatiana I am using WASB connection details to connect to ADLS gen2 storage account, apart the connection details nothing is similar I think



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1058797680


##########
tests/providers/microsoft/azure/hooks/test_adls_v2.py:
##########
@@ -0,0 +1,108 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   @eladkal Can you review it now



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1051823169


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):
+
+    conn_name_attr = "adls_v2_conn_id"
+    default_conn_name = "adls_v2_default"
+    conn_type = "adls_v2"
+    hook_name = "Azure Date Lake Storage"
+
+    @staticmethod
+    def get_connection_form_widgets() -> dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3PasswordFieldWidget, BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import PasswordField, StringField
+
+        return {
+            "extra__adls_v2__connection_string": PasswordField(
+                lazy_gettext("Blob Storage Connection String (optional)"), widget=BS3PasswordFieldWidget()
+            ),
+            "extra__adls_v2__tenant_id": StringField(
+                lazy_gettext("Tenant Id (Active Directory Auth)"), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ["schema", "port"],
+            "relabeling": {
+                "login": "Blob Storage Login (optional)",
+                "password": "Blob Storage Key (optional)",
+                "host": "Account Name (Active Directory Auth)",
+            },
+            "placeholders": {
+                "login": "account name",
+                "password": "secret",
+                "host": "account url",
+                "extra__adls_v2__connection_string": "connection string auth",
+                "extra__adls_v2__tenant_id": "tenant",
+            },
+        }
+
+    def __init__(self, adls_v2_conn_id: str = default_conn_name, public_read: bool = False) -> None:
+        super().__init__()
+        self.conn_id = adls_v2_conn_id
+        self.public_read = public_read
+        self.service_client = self.get_conn()
+
+    def get_conn(self) -> DataLakeServiceClient:
+        """Return the DataLakeServiceClient object."""
+        conn = self.get_connection(self.conn_id)
+        extra = conn.extra_dejson or {}
+
+        connection_string = extra.pop(
+            "connection_string", extra.pop("extra__adls_v2__connection_string", None)
+        )
+        if connection_string:
+            # connection_string auth takes priority
+            return DataLakeServiceClient.from_connection_string(connection_string, **extra)
+
+        tenant = extra.pop("tenant_id", extra.pop("extra__adls_v2__tenant_id", None))
+        if tenant:
+            # use Active Directory auth
+            app_id = conn.login
+            app_secret = conn.password
+            token_credential = ClientSecretCredential(tenant, app_id, app_secret)
+            return DataLakeServiceClient(
+                account_url=f"https://{conn.login}.dfs.core.windows.net", credential=token_credential, **extra
+            )
+        credential = conn.password

Review Comment:
   @tatiana ADLS gen2 is using different protocols and APIs to connect. Currently I have created separate connection type and connection details.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] kaxil commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
kaxil commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1054919868


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):

Review Comment:
   @bharanidharan14 ^^



-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049543186


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   If wasb is legacy and we expect it to be removed some day then I don't think we utlize it for other services
   but I'm not Azure expert - I leave this desicion to people who knows the service.
   
   What i do want to have here is docs and specifically connection docs that explains how to make this work.
   (I want to avoid questions raised down the road by users who will ask to refactor because we utlize legacy service)



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1051820566


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   @luanmorenomaciel I don't think inheriting from the **airflow.providers.microsoft.azure.hooks.azure_data_lake** is also not recommended because this ADLS existing hook is gen1 and it's being getting retired from feb. They may stop supporting for that too as well. so its better to inherit from `BaseHook`. WDYT ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] tatiana commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1056265305


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):
+    """
+    This Hook interacts with ADLS gen2 storage account it mainly helps to create and manage
+    directories and files in storage accounts that have a hierarchical namespace. Using Alds_v2 connection
+    details create DataLakeServiceClient object
+
+    Due to Wasb is marked as legacy and and retirement of the (ADLS1) it would be nice to
+    implement ALDS gen2 hook for interacting with the storage account.
+
+    .. seealso::
+        https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-directory-file-acl-python
+
+    :param adls_v2_conn_id: Reference to the :ref:`adls_v2 connection <howto/connection:adls_v2>`.

Review Comment:
   Hi @bharanidharan14 , it would be great if we could rename things as follows:
   
   * `airflow/providers/microsoft/azure/hooks/adls_v2.py` -> `airflow/providers/microsoft/azure/hooks/azure_data_lake.py` (use existing module)
   * `AdlsClientHook` -> AzureDataLakeStorageV2Hook` (be more explicit since we'll have V1 and V2 ADLS hooks coexisting for some time in Airflow/Azure, it is probably worth having V2 explicit in the class name)
   * `adls_v2_conn_id` -> `adls_conn_id` (if the class name has V2, we can document that this parameter should be a V2 connection ID, but I think the parameter could be called only adls_conn_id)



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049253889


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   Now I changed the Hook name now.



-- 
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] tatiana commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1050879716


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   @bharanidharan14 @eladkal  did we consider having an `AzureBaseHook` similar to `GoogleBaseHook`
   https://github.com/apache/airflow/blob/main/airflow/providers/google/common/hooks/base_google.py
   This way, both `AzureDataLakeStorageHook` and `WasbHook` could inherit from it



-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1057634421


##########
tests/providers/microsoft/azure/hooks/test_adls_v2.py:
##########
@@ -0,0 +1,108 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   If the hook logic is in `hooks/data_lake.py` why do we create a `test_adls_v2.py` ?



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1057541197


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):
+    """
+    This Hook interacts with ADLS gen2 storage account it mainly helps to create and manage
+    directories and files in storage accounts that have a hierarchical namespace. Using Alds_v2 connection
+    details create DataLakeServiceClient object
+
+    Due to Wasb is marked as legacy and and retirement of the (ADLS1) it would be nice to
+    implement ALDS gen2 hook for interacting with the storage account.
+
+    .. seealso::
+        https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-directory-file-acl-python
+
+    :param adls_v2_conn_id: Reference to the :ref:`adls_v2 connection <howto/connection:adls_v2>`.

Review Comment:
   @tatiana Will address this and let you know 



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1058690039


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):

Review Comment:
   Addressed it 



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049530878


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   Only for connection details, should I create new connection ?



-- 
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] tatiana commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1050892506


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):
+
+    conn_name_attr = "adls_v2_conn_id"
+    default_conn_name = "adls_v2_default"
+    conn_type = "adls_v2"
+    hook_name = "Azure Date Lake Storage"
+
+    @staticmethod
+    def get_connection_form_widgets() -> dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3PasswordFieldWidget, BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import PasswordField, StringField
+
+        return {
+            "extra__adls_v2__connection_string": PasswordField(
+                lazy_gettext("Blob Storage Connection String (optional)"), widget=BS3PasswordFieldWidget()
+            ),
+            "extra__adls_v2__tenant_id": StringField(
+                lazy_gettext("Tenant Id (Active Directory Auth)"), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ["schema", "port"],
+            "relabeling": {
+                "login": "Blob Storage Login (optional)",
+                "password": "Blob Storage Key (optional)",
+                "host": "Account Name (Active Directory Auth)",
+            },
+            "placeholders": {
+                "login": "account name",
+                "password": "secret",
+                "host": "account url",
+                "extra__adls_v2__connection_string": "connection string auth",
+                "extra__adls_v2__tenant_id": "tenant",
+            },
+        }
+
+    def __init__(self, adls_v2_conn_id: str = default_conn_name, public_read: bool = False) -> None:
+        super().__init__()
+        self.conn_id = adls_v2_conn_id
+        self.public_read = public_read
+        self.service_client = self.get_conn()
+
+    def get_conn(self) -> DataLakeServiceClient:
+        """Return the DataLakeServiceClient object."""
+        conn = self.get_connection(self.conn_id)
+        extra = conn.extra_dejson or {}
+
+        connection_string = extra.pop(
+            "connection_string", extra.pop("extra__adls_v2__connection_string", None)
+        )
+        if connection_string:
+            # connection_string auth takes priority
+            return DataLakeServiceClient.from_connection_string(connection_string, **extra)
+
+        tenant = extra.pop("tenant_id", extra.pop("extra__adls_v2__tenant_id", None))
+        if tenant:
+            # use Active Directory auth
+            app_id = conn.login
+            app_secret = conn.password
+            token_credential = ClientSecretCredential(tenant, app_id, app_secret)
+            return DataLakeServiceClient(
+                account_url=f"https://{conn.login}.dfs.core.windows.net", credential=token_credential, **extra
+            )
+        credential = conn.password

Review Comment:
   That's a shame, @bharanidharan14 - I didn't realise things were so disconnected in Azure



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1058797784


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):
+    """
+    This Hook interacts with ADLS gen2 storage account it mainly helps to create and manage
+    directories and files in storage accounts that have a hierarchical namespace. Using Alds_v2 connection
+    details create DataLakeServiceClient object
+
+    Due to Wasb is marked as legacy and and retirement of the (ADLS1) it would be nice to
+    implement ALDS gen2 hook for interacting with the storage account.
+
+    .. seealso::
+        https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-directory-file-acl-python
+
+    :param adls_v2_conn_id: Reference to the :ref:`adls_v2 connection <howto/connection:adls_v2>`.

Review Comment:
   @tatiana Can you review it now



-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049552123


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   since we don't have hook for adls it can be anything we want.
   i suggested the pattern: `Adls*` so:
   
   
   | Current V1         | Suggested V2       |
   |--------------------|--------------------|
   |          -         | AdlsClientHook     |
   | ADLSDeleteOperator | AdlsDeleteOperator |
   | ADLSListOperator   | AdlsListOperator   |
   



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049530878


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   Only for connection details, I thought of using these WASB connection details. should I create a new connection ?



-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049516430


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   https://github.com/apache/airflow/issues/28223 description states that wasb is legacy if so what is the motivation to inhert from wsab for adsl gen2?



-- 
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] tatiana commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1046543896


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):
+
+    conn_name_attr = "adls_v2_conn_id"
+    default_conn_name = "adls_v2_default"
+    conn_type = "adls_v2"
+    hook_name = "Azure Date Lake Storage"
+
+    @staticmethod
+    def get_connection_form_widgets() -> dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3PasswordFieldWidget, BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import PasswordField, StringField
+
+        return {
+            "extra__adls_v2__connection_string": PasswordField(
+                lazy_gettext("Blob Storage Connection String (optional)"), widget=BS3PasswordFieldWidget()
+            ),
+            "extra__adls_v2__tenant_id": StringField(
+                lazy_gettext("Tenant Id (Active Directory Auth)"), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ["schema", "port"],
+            "relabeling": {
+                "login": "Blob Storage Login (optional)",
+                "password": "Blob Storage Key (optional)",
+                "host": "Account Name (Active Directory Auth)",
+            },
+            "placeholders": {
+                "login": "account name",
+                "password": "secret",
+                "host": "account url",
+                "extra__adls_v2__connection_string": "connection string auth",
+                "extra__adls_v2__tenant_id": "tenant",
+            },
+        }
+
+    def __init__(self, adls_v2_conn_id: str = default_conn_name, public_read: bool = False) -> None:
+        super().__init__()
+        self.conn_id = adls_v2_conn_id
+        self.public_read = public_read
+        self.service_client = self.get_conn()
+
+    def get_conn(self) -> DataLakeServiceClient:
+        """Return the DataLakeServiceClient object."""
+        conn = self.get_connection(self.conn_id)
+        extra = conn.extra_dejson or {}
+
+        connection_string = extra.pop(
+            "connection_string", extra.pop("extra__adls_v2__connection_string", None)
+        )
+        if connection_string:
+            # connection_string auth takes priority
+            return DataLakeServiceClient.from_connection_string(connection_string, **extra)
+
+        tenant = extra.pop("tenant_id", extra.pop("extra__adls_v2__tenant_id", None))
+        if tenant:
+            # use Active Directory auth
+            app_id = conn.login
+            app_secret = conn.password
+            token_credential = ClientSecretCredential(tenant, app_id, app_secret)
+            return DataLakeServiceClient(
+                account_url=f"https://{conn.login}.dfs.core.windows.net", credential=token_credential, **extra
+            )
+        credential = conn.password

Review Comment:
   Would it make sense to do something similar to:
   https://github.com/apache/airflow/blob/main/airflow/providers/microsoft/azure/hooks/wasb.py#L202?
   
   I wonder how much in common the Wasb & the AzureDataLakeStorageV2 implementation have - and if it would make sense to have a `BaseAzureHook`



##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   ```suggestion
   class AzureDataLakeStorageV2Hook(BaseHook):
   ```



-- 
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] luanmorenomaciel commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
luanmorenomaciel commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1050905172


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   @tatiana and @mateusoliveiraowshq as per our conversation, implementing the `AzureBaseHook` would be challenging since Azure unfortunately does not share the same connection client principles (BlobServiceClient & DataLakeServiceClient) between the WASB and ABFS protocol.
   
   [Azure Blob Storage - WASB ](https://pypi.org/project/azure-storage-blob/)
   [Azure Data Lake Storage Gen2 - ABFS](https://pypi.org/project/azure-storage-file-datalake/)
   
   What we could do instead of is to have the following hook structure:
   
   [airflow.providers.microsoft.azure.hooks.wasb](https://airflow.apache.org/docs/apache-airflow-providers-microsoft-azure/stable/_api/airflow/providers/microsoft/azure/hooks/wasb/index.html) (Already Exists)
   
   [airflow.providers.microsoft.azure.hooks.azure_data_lake](https://airflow.apache.org/docs/apache-airflow-providers-microsoft-azure/1.0.0/_api/airflow/providers/microsoft/azure/hooks/azure_data_lake/index.html#module-airflow.providers.microsoft.azure.hooks.azure_data_lake) (Suggested)
   
   Meaning that as per @bharanidharan14 @eladkal conversation we would recommend to inherid the hook from the **airflow.providers.microsoft.azure.hooks.azure_data_lake** instead of **airflow.providers.microsoft.azure.hooks.wasb**, that would make more sense since WASB protocol has been marked as legacy hence the, implementation process are different from each other.
   
   
   
   ![wasb-abfs](https://user-images.githubusercontent.com/20648427/208137163-382b616f-d14b-413c-afa6-0c7506db95e6.png)
   
   



-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049522681


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   I'm not sure about this.
   eventually v2 will be the standard and on that day it would be redundant to mention v2.
   
   I see that all current v1 services are listed as `AzueDataLakeStorage*`
   WDYT about having the new ones as `Adls*`?



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049545222


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   Ok let me add New connection and with docs 



-- 
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] bharanidharan14 commented on pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on PR #28262:
URL: https://github.com/apache/airflow/pull/28262#issuecomment-1358906988

   @kaxil Need your review on this PR


-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1058689769


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):
+    """
+    This Hook interacts with ADLS gen2 storage account it mainly helps to create and manage
+    directories and files in storage accounts that have a hierarchical namespace. Using Alds_v2 connection
+    details create DataLakeServiceClient object
+
+    Due to Wasb is marked as legacy and and retirement of the (ADLS1) it would be nice to
+    implement ALDS gen2 hook for interacting with the storage account.
+
+    .. seealso::
+        https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-directory-file-acl-python
+
+    :param adls_v2_conn_id: Reference to the :ref:`adls_v2 connection <howto/connection:adls_v2>`.

Review Comment:
   @tatiana Addressed your suggestion 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1053374310


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   @luanmorenomaciel Currently I am inheriting from  BaseHook. Can you also review this PR 



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049533584


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   Yes, the current  v1 services are listed as AzueDataLakeStorage. Now it is `AzureDataLakeStorageClient` can it be `AzureDataLakeStorageClientHook`. WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] bharanidharan14 commented on pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on PR #28262:
URL: https://github.com/apache/airflow/pull/28262#issuecomment-1348796272

   @tatiana @luanmorenomaciel Need your review on this PR.


-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049545222


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,264 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+
+from airflow.providers.microsoft.azure.hooks.wasb import WasbHook
+
+
+class AzureDataLakeStorageClient(WasbHook):

Review Comment:
   Ok let me add new connection, with docs 



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1057642787


##########
tests/providers/microsoft/azure/hooks/test_adls_v2.py:
##########
@@ -0,0 +1,108 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Yeah @eladkal I am moving the tests to `test_azure_data_lake` file. working on it



-- 
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] tatiana commented on pull request #28262: [WIP]: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on PR #28262:
URL: https://github.com/apache/airflow/pull/28262#issuecomment-1347485715

   Relates to #28223 


-- 
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 pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
kaxil commented on PR #28262:
URL: https://github.com/apache/airflow/pull/28262#issuecomment-1362240848

   Tests are failing


-- 
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 #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1049554650


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   Alternatively we can also do:
   microsoft/azure/hooks/adsl_v2.py
   microsoft/azure/operators/adsl_v2.py
   then class names can have the same names as v1 because they are not in the same file
   and once v1 is removed we can change the files names which is easier change then changing classes names.
   



##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):

Review Comment:
   Alternatively we can also do:
   ```
   microsoft/azure/hooks/adsl_v2.py
   microsoft/azure/operators/adsl_v2.py
   ```
   then class names can have the same names as v1 because they are not in the same file
   and once v1 is removed we can change the files names which is easier change then changing classes names.
   



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1051823169


##########
airflow/providers/microsoft/azure/hooks/adls.py:
##########
@@ -0,0 +1,221 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    FileSystemClient,
+)
+
+from airflow.hooks.base import BaseHook
+
+
+class AzureDataLakeStorageV2(BaseHook):
+
+    conn_name_attr = "adls_v2_conn_id"
+    default_conn_name = "adls_v2_default"
+    conn_type = "adls_v2"
+    hook_name = "Azure Date Lake Storage"
+
+    @staticmethod
+    def get_connection_form_widgets() -> dict[str, Any]:
+        """Returns connection widgets to add to connection form"""
+        from flask_appbuilder.fieldwidgets import BS3PasswordFieldWidget, BS3TextFieldWidget
+        from flask_babel import lazy_gettext
+        from wtforms import PasswordField, StringField
+
+        return {
+            "extra__adls_v2__connection_string": PasswordField(
+                lazy_gettext("Blob Storage Connection String (optional)"), widget=BS3PasswordFieldWidget()
+            ),
+            "extra__adls_v2__tenant_id": StringField(
+                lazy_gettext("Tenant Id (Active Directory Auth)"), widget=BS3TextFieldWidget()
+            ),
+        }
+
+    @staticmethod
+    def get_ui_field_behaviour() -> dict[str, Any]:
+        """Returns custom field behaviour"""
+        return {
+            "hidden_fields": ["schema", "port"],
+            "relabeling": {
+                "login": "Blob Storage Login (optional)",
+                "password": "Blob Storage Key (optional)",
+                "host": "Account Name (Active Directory Auth)",
+            },
+            "placeholders": {
+                "login": "account name",
+                "password": "secret",
+                "host": "account url",
+                "extra__adls_v2__connection_string": "connection string auth",
+                "extra__adls_v2__tenant_id": "tenant",
+            },
+        }
+
+    def __init__(self, adls_v2_conn_id: str = default_conn_name, public_read: bool = False) -> None:
+        super().__init__()
+        self.conn_id = adls_v2_conn_id
+        self.public_read = public_read
+        self.service_client = self.get_conn()
+
+    def get_conn(self) -> DataLakeServiceClient:
+        """Return the DataLakeServiceClient object."""
+        conn = self.get_connection(self.conn_id)
+        extra = conn.extra_dejson or {}
+
+        connection_string = extra.pop(
+            "connection_string", extra.pop("extra__adls_v2__connection_string", None)
+        )
+        if connection_string:
+            # connection_string auth takes priority
+            return DataLakeServiceClient.from_connection_string(connection_string, **extra)
+
+        tenant = extra.pop("tenant_id", extra.pop("extra__adls_v2__tenant_id", None))
+        if tenant:
+            # use Active Directory auth
+            app_id = conn.login
+            app_secret = conn.password
+            token_credential = ClientSecretCredential(tenant, app_id, app_secret)
+            return DataLakeServiceClient(
+                account_url=f"https://{conn.login}.dfs.core.windows.net", credential=token_credential, **extra
+            )
+        credential = conn.password

Review Comment:
   @tatiana ADLS gen2 is using different protocols and APIs and different client-type principles to connect. Currently, I have created separate connection types and connection details.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] tatiana commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
tatiana commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1050906637


##########
airflow/providers/microsoft/azure/hooks/adls_v2.py:
##########
@@ -0,0 +1,307 @@
+# 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.
+
+from __future__ import annotations
+
+from typing import Any
+
+from azure.core.exceptions import ResourceExistsError, ResourceNotFoundError
+from azure.identity import ClientSecretCredential
+from azure.storage.filedatalake import (
+    DataLakeDirectoryClient,
+    DataLakeFileClient,
+    DataLakeServiceClient,
+    DirectoryProperties,
+    FileSystemClient,
+    FileSystemProperties,
+)
+from hooks.base import BaseHook
+
+
+class AdlsClientHook(BaseHook):

Review Comment:
   Would it make sense for this to be defined inside `airflow/providers/microsoft/azure/hooks/azure_data_lake`?



-- 
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] bharanidharan14 commented on a diff in pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
bharanidharan14 commented on code in PR #28262:
URL: https://github.com/apache/airflow/pull/28262#discussion_r1058689964


##########
tests/providers/microsoft/azure/hooks/test_adls_v2.py:
##########
@@ -0,0 +1,108 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Moved the test to `test_azure_data_lake` 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] kaxil merged pull request #28262: Hook for managing directories and files in Azure Data Lake Storage Gen2

Posted by GitBox <gi...@apache.org>.
kaxil merged PR #28262:
URL: https://github.com/apache/airflow/pull/28262


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