Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
48 commits
Select commit Hold shift + click to select a range
c89fbfb
feat: Pass proxies config when using ClientSecretCredential in AzureD…
Jan 30, 2024
754a5f1
refactor: Try fixing typing issues
davidblain-infrabel Jan 31, 2024
0469dd9
docs: Updated provider docs
davidblain-infrabel Jan 31, 2024
78e7fa2
refactor: Reformatted files
davidblain-infrabel Jan 31, 2024
e6f142c
refactor: Make sure upload_data method is verified and asserted
davidblain-infrabel Jan 31, 2024
f50dfd1
refactor: Reformatted some files
davidblain-infrabel Jan 31, 2024
62753a4
refactor: Try fixing mypy issues
davidblain-infrabel Jan 31, 2024
55f3ba3
refactor: Fixed additional static checks
davidblain-infrabel Jan 31, 2024
9127631
refactor: Fixed length parameter type definition in operator
davidblain-infrabel Jan 31, 2024
7374c47
refactor: Removed unused import
davidblain-infrabel Jan 31, 2024
51b319d
docs: Updated docstring for DataToADLSOperator
davidblain-infrabel Feb 8, 2024
e8c0638
Merge branch 'main' into fix/azure-data-lake-proxies
dabla Feb 29, 2024
8124a9d
Merge branch 'main' into fix/azure-data-lake-proxies
dabla Feb 29, 2024
be95816
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 1, 2024
5f520ab
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 1, 2024
7517de9
refactor: Renamed DataToADLSOperator to ADLSCreateObjectOperator and …
davidblain-infrabel Mar 4, 2024
90d5dd7
fix: Re-added wrongly removed unit test
davidblain-infrabel Mar 4, 2024
490ef76
refactor: Re-ordered typing imports in adls module
davidblain-infrabel Mar 4, 2024
affca82
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 4, 2024
6dca248
docs: Replaced tiles with dashes
davidblain-infrabel Mar 4, 2024
c983f8a
docs: Fixed class reference to ADLSDeleteOperator
davidblain-infrabel Mar 4, 2024
7a0003e
docs: Updated comment near class reference for ADLSCreateObjectOperator
davidblain-infrabel Mar 4, 2024
3eaeb6e
docs: Added example of ADLSListOperator
davidblain-infrabel Mar 4, 2024
67149ad
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 5, 2024
4be01cf
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 5, 2024
ec4c84e
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 6, 2024
65b2f74
fix: Reverted wrong rename of replace to overwrite due to refactoring…
Mar 5, 2024
31c494b
refactor: Reformatted import of LocalFilesystemToADLSOperator as in m…
davidblain-infrabel Mar 6, 2024
ccd1f20
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 6, 2024
de9d654
fix: Fixed patch of AzureDataLakeStorageV2Hook in TestADLSUploadOperator
davidblain-infrabel Mar 6, 2024
edf14c0
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 6, 2024
309eadc
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 7, 2024
1c45e9c
Update tests/system/providers/microsoft/azure/example_adls_list.py
dabla Mar 7, 2024
36108b5
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 7, 2024
d3e5729
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 8, 2024
8e8858f
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 8, 2024
8f08da0
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 11, 2024
a3753e9
docs: Temporary removed docs for ADLSListOperator
davidblain-infrabel Mar 11, 2024
a4fda00
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 11, 2024
a3de8fb
Revert "docs: Temporary removed docs for ADLSListOperator"
davidblain-infrabel Mar 11, 2024
77f6cae
fix: Fixed unit tests for TestADLSUploadOperator
davidblain-infrabel Mar 12, 2024
7bcb075
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 12, 2024
346112c
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 13, 2024
66f1ed5
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 13, 2024
ec32733
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 13, 2024
49b50cf
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 14, 2024
1545c84
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 15, 2024
8051b3d
Merge branch 'main' into feature/data-to-adls-operator
dabla Mar 15, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
83 changes: 64 additions & 19 deletions airflow/providers/microsoft/azure/operators/adls.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,22 +16,74 @@
# under the License.
from __future__ import annotations

from typing import TYPE_CHECKING, Any, Sequence
from typing import IO, TYPE_CHECKING, Any, AnyStr, Iterable, Sequence

from airflow.models import BaseOperator
from airflow.providers.microsoft.azure.hooks.data_lake import AzureDataLakeHook
from airflow.providers.microsoft.azure.hooks.data_lake import AzureDataLakeHook, AzureDataLakeStorageV2Hook

if TYPE_CHECKING:
from airflow.utils.context import Context

DEFAULT_AZURE_DATA_LAKE_CONN_ID = "azure_data_lake_default"


class ADLSCreateObjectOperator(BaseOperator):
"""
Creates a new object from passed data to Azure Data Lake on specified file.

.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:ADLSCreateObjectOperator`

:param file_system_name: Name of the file system or instance of FileSystemProperties.
:param file_name: Name of the file which needs to be created in the file system.
:param data: The data that will be uploaded.
:param length: Size of the data in bytes (optional).
:param replace: Whether to forcibly overwrite existing files/directories.
If False and remote path is a directory, will quit regardless if any files
would be overwritten or not. If True, only matching filenames are actually
overwritten.
:param azure_data_lake_conn_id: Reference to the :ref:`Azure Data Lake connection<howto/connection:adl>`.
"""

template_fields: Sequence[str] = ("file_system_name", "file_name", "data")
ui_color = "#e4f0e8"

def __init__(
self,
*,
file_system_name: str,
file_name: str,
data: bytes | str | Iterable[AnyStr] | IO[AnyStr],
length: int | None = None,
replace: bool = False,
azure_data_lake_conn_id: str = DEFAULT_AZURE_DATA_LAKE_CONN_ID,
**kwargs,
) -> None:
super().__init__(**kwargs)

self.file_system_name = file_system_name
self.file_name = file_name
self.replace = replace
self.data = data # type: ignore[var-annotated]
self.length = length
self.azure_data_lake_conn_id = azure_data_lake_conn_id

def execute(self, context: Context) -> dict[str, Any]:
self.log.debug("Uploading %s to %s", self.data, self.file_name)
hook = AzureDataLakeStorageV2Hook(adls_conn_id=self.azure_data_lake_conn_id)
return hook.create_file(file_system_name=self.file_system_name, file_name=self.file_name).upload_data(
data=self.data, length=self.length, overwrite=self.replace
)


class ADLSDeleteOperator(BaseOperator):
"""
Delete files in the specified path.

.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:ADLSDeleteOperator`
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:ADLSDeleteOperator`

:param path: A directory or file to remove
:param recursive: Whether to loop into directories in the location and remove the files
Expand All @@ -48,7 +100,7 @@ def __init__(
path: str,
recursive: bool = False,
ignore_not_found: bool = True,
azure_data_lake_conn_id: str = "azure_data_lake_default",
azure_data_lake_conn_id: str = DEFAULT_AZURE_DATA_LAKE_CONN_ID,
**kwargs,
) -> None:
super().__init__(**kwargs)
Expand All @@ -69,26 +121,19 @@ class ADLSListOperator(BaseOperator):
This operator returns a python list with the names of files which can be used by
`xcom` in the downstream tasks.

:param path: The Azure Data Lake path to find the objects. Supports glob
strings (templated)
:param azure_data_lake_conn_id: Reference to the :ref:`Azure Data Lake connection<howto/connection:adl>`.
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:ADLSListOperator`

**Example**:
The following Operator would list all the Parquet files from ``folder/output/``
folder in the specified ADLS account ::

adls_files = ADLSListOperator(
task_id="adls_files",
path="folder/output/*.parquet",
azure_data_lake_conn_id="azure_data_lake_default",
)
:param path: The Azure Data Lake path to find the objects. Supports glob strings (templated)
:param azure_data_lake_conn_id: Reference to the :ref:`Azure Data Lake connection<howto/connection:adl>`.
"""

template_fields: Sequence[str] = ("path",)
ui_color = "#901dd2"

def __init__(
self, *, path: str, azure_data_lake_conn_id: str = "azure_data_lake_default", **kwargs
self, *, path: str, azure_data_lake_conn_id: str = DEFAULT_AZURE_DATA_LAKE_CONN_ID, **kwargs
) -> None:
super().__init__(**kwargs)
self.path = path
Expand Down
36 changes: 35 additions & 1 deletion docs/apache-airflow-providers-microsoft-azure/operators/adls.rst
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,29 @@ Prerequisite Tasks

.. include:: /operators/_partials/prerequisite_tasks.rst

.. _howto/operator:ADLSCreateObjectOperator:

ADLSCreateObjectOperator
----------------------------------

:class:`~airflow.providers.microsoft.azure.operators.adls.ADLSCreateObjectOperator` allows you to
upload data to Azure DataLake Storage


Below is an example of using this operator to upload data to ADL.

.. exampleinclude:: /../../tests/system/providers/microsoft/azure/example_adls_create.py
:language: python
:dedent: 0
:start-after: [START howto_operator_adls_create]
:end-before: [END howto_operator_adls_create]

.. _howto/operator:ADLSDeleteOperator:

ADLSDeleteOperator
----------------------------------
Use the
:class:`~airflow.providers.microsoft.azure.operators.adls_delete.ADLSDeleteOperator` to remove
:class:`~airflow.providers.microsoft.azure.operators.adls.ADLSDeleteOperator` to remove
file(s) from Azure DataLake Storage


Expand All @@ -41,6 +58,23 @@ Below is an example of using this operator to delete a file from ADL.
:start-after: [START howto_operator_adls_delete]
:end-before: [END howto_operator_adls_delete]

.. _howto/operator:ADLSListOperator:

ADLSListOperator
----------------------------------
Use the
:class:`~airflow.providers.microsoft.azure.operators.adls.ADLSListOperator` to list all
file(s) from Azure DataLake Storage


Below is an example of using this operator to list files from ADL.

.. exampleinclude:: /../../tests/system/providers/microsoft/azure/example_adls_list.py
:language: python
:dedent: 0
:start-after: [START howto_operator_adls_list]
:end-before: [END howto_operator_adls_list]


Reference
---------
Expand Down
47 changes: 47 additions & 0 deletions tests/providers/microsoft/azure/operators/test_adls_create.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
#
# 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

import json
from unittest import mock

from airflow.providers.microsoft.azure.operators.adls import ADLSCreateObjectOperator

TASK_ID = "test-adls-upload-operator"
FILE_SYSTEM_NAME = "Fabric"
REMOTE_PATH = "TEST-DIR"
DATA = json.dumps({"name": "David", "surname": "Blain", "gender": "M"}).encode("utf-8")


class TestADLSUploadOperator:
@mock.patch("airflow.providers.microsoft.azure.operators.adls.AzureDataLakeStorageV2Hook")
def test_execute_success_when_local_data(self, mock_hook):
operator = ADLSCreateObjectOperator(
task_id=TASK_ID,
file_system_name=FILE_SYSTEM_NAME,
file_name=REMOTE_PATH,
data=DATA,
replace=True,
)
operator.execute(None)
data_lake_file_client_mock = mock_hook.return_value.create_file
data_lake_file_client_mock.assert_called_once_with(
file_system_name=FILE_SYSTEM_NAME, file_name=REMOTE_PATH
)
upload_data_mock = data_lake_file_client_mock.return_value.upload_data
upload_data_mock.assert_called_once_with(data=DATA, length=None, overwrite=True)
Original file line number Diff line number Diff line change
Expand Up @@ -17,22 +17,17 @@
# under the License.
from __future__ import annotations

import json
from unittest import mock

import pytest

from airflow.exceptions import AirflowException
from airflow.providers.microsoft.azure.transfers.local_to_adls import (
LocalFilesystemToADLSOperator,
)
from airflow.providers.microsoft.azure.transfers.local_to_adls import LocalFilesystemToADLSOperator

TASK_ID = "test-adls-upload-operator"
FILE_SYSTEM_NAME = "Fabric"
LOCAL_PATH = "test/*"
BAD_LOCAL_PATH = "test/**"
REMOTE_PATH = "TEST-DIR"
DATA = json.dumps({"name": "David", "surname": "Blain", "gender": "M"}).encode("utf-8")


class TestADLSUploadOperator:
Expand Down
58 changes: 58 additions & 0 deletions tests/system/providers/microsoft/azure/example_adls_create.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
# 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

import os
from datetime import datetime

from airflow import models
from airflow.providers.microsoft.azure.operators.adls import ADLSCreateObjectOperator, ADLSDeleteOperator

REMOTE_FILE_PATH = os.environ.get("REMOTE_LOCAL_PATH", "remote.txt")
DAG_ID = "example_adls_create"

with models.DAG(
DAG_ID,
start_date=datetime(2021, 1, 1),
catchup=False,
schedule=None,
tags=["example"],
) as dag:
# [START howto_operator_adls_create]
upload_data = ADLSCreateObjectOperator(
task_id="upload_data",
file_system_name="Fabric",
file_name=REMOTE_FILE_PATH,
data="Hello world",
replace=True,
)
# [END howto_operator_adls_create]

delete_file = ADLSDeleteOperator(task_id="remove_task", path=REMOTE_FILE_PATH, recursive=True)

upload_data >> delete_file

from tests.system.utils.watcher import watcher

# This test needs watcher in order to properly mark success/failure
# when "tearDown" task with trigger rule is part of the DAG
list(dag.tasks) >> watcher()

from tests.system.utils import get_test_run # noqa: E402

# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest)
test_run = get_test_run(dag)
54 changes: 54 additions & 0 deletions tests/system/providers/microsoft/azure/example_adls_list.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
# 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

import os
from datetime import datetime

from airflow import models
from airflow.providers.microsoft.azure.operators.adls import ADLSListOperator

LOCAL_FILE_PATH = os.environ.get("LOCAL_FILE_PATH", "localfile.txt")
REMOTE_FILE_PATH = os.environ.get("REMOTE_LOCAL_PATH", "remote.txt")

ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
DAG_ID = "example_adls_list"

with models.DAG(
DAG_ID,
start_date=datetime(2021, 1, 1),
schedule=None,
tags=["example"],
) as dag:
# [START howto_operator_adls_list]
adls_files = ADLSListOperator(
task_id="adls_files",
path="folder/output/*.parquet",
azure_data_lake_conn_id="azure_data_lake_default",
)
# [END howto_operator_adls_list]

from tests.system.utils.watcher import watcher

# This test needs watcher in order to properly mark success/failure
# when "tearDown" task with trigger rule is part of the DAG
list(dag.tasks) >> watcher()

from tests.system.utils import get_test_run # noqa: E402

# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest)
test_run = get_test_run(dag)