Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Teach Azure Filesystem to authenticate using DefaultAzureCredential in the Python SDK #24212

Merged
merged 16 commits into from
Nov 22, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
* Support for Bigtable sink (Write and WriteBatch) added (Go) ([#23324](https://github.com/apache/beam/issues/23324)).
* S3 implementation of the Beam filesystem (Go) ([#23991](https://github.com/apache/beam/issues/23991)).
* Support for SingleStoreDB source and sink added (Java) ([#22617](https://github.com/apache/beam/issues/22617)).
* Added support for DefaultAzureCredential authentication in Azure Filesystem (Python) ([#24210](https://github.com/apache/beam/issues/24210)).

## New Features / Improvements

Expand Down
1 change: 1 addition & 0 deletions build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -340,6 +340,7 @@ tasks.register("python37PostCommit") {
dependsOn(":sdks:python:test-suites:direct:py37:postCommitIT")
dependsOn(":sdks:python:test-suites:direct:py37:directRunnerIT")
dependsOn(":sdks:python:test-suites:direct:py37:hdfsIntegrationTest")
dependsOn(":sdks:python:test-suites:direct:py37:azureIntegrationTest")
dependsOn(":sdks:python:test-suites:direct:py37:mongodbioIT")
dependsOn(":sdks:python:test-suites:portable:py37:postCommitPy37")
dependsOn(":sdks:python:test-suites:dataflow:py37:spannerioIT")
Expand Down
18 changes: 18 additions & 0 deletions sdks/python/apache_beam/internal/azure/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
#
# 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.
#

"""For internal use only; no backwards-compatibility guarantees."""
83 changes: 83 additions & 0 deletions sdks/python/apache_beam/internal/azure/auth.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
#
# 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.
#

"""Azure credentials and authentication."""

# pytype: skip-file

import logging
import threading

from apache_beam.options.pipeline_options import AzureOptions

try:
from azure.identity import DefaultAzureCredential
_AZURE_AUTH_AVAILABLE = True
except ImportError:
_AZURE_AUTH_AVAILABLE = False

_LOGGER = logging.getLogger(__name__)


def get_service_credentials(pipeline_options):
"""For internal use only; no backwards-compatibility guarantees.

Get credentials to access Azure services.
Args:
pipeline_options: Pipeline options, used in creating credentials
like managed identity credentials.

Returns:
A ``azure.identity.*Credential`` object or None if credentials
not found. Returned object is thread-safe.
"""
return _Credentials.get_service_credentials(pipeline_options)


class _Credentials(object):
_credentials_lock = threading.Lock()
_credentials_init = False
_credentials = None

@classmethod
def get_service_credentials(cls, pipeline_options):
with cls._credentials_lock:
if cls._credentials_init:
return cls._credentials
cls._credentials = cls._get_service_credentials(pipeline_options)
cls._credentials_init = True

return cls._credentials

@staticmethod
def _get_service_credentials(pipeline_options):
if not _AZURE_AUTH_AVAILABLE:
_LOGGER.warning(
'Unable to find default credentials because the azure.identity '
'library is not available. Install the azure.identity library to use '
'Azure default credentials.')
return None

try:
credentials = DefaultAzureCredential(
managed_identity_client_id=pipeline_options.view_as(AzureOptions)\
.azure_managed_identity_client_id)
_LOGGER.debug('Connecting using Azure Default Credentials.')
return credentials
except Exception as e:
_LOGGER.warning('Unable to find Azure credentials to use: %s\n', e)
return None
28 changes: 17 additions & 11 deletions sdks/python/apache_beam/io/azure/blobstoragefilesystem.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,10 @@ class BlobStorageFileSystem(FileSystem):
CHUNK_SIZE = blobstorageio.MAX_BATCH_OPERATION_SIZE
AZURE_FILE_SYSTEM_PREFIX = 'azfs://'

def __init__(self, pipeline_options):
super().__init__(pipeline_options)
self._pipeline_options = pipeline_options

@classmethod
def scheme(cls):
"""URI scheme for the FileSystem
Expand Down Expand Up @@ -118,12 +122,15 @@ def _list(self, dir_or_prefix):
``BeamIOError``: if listing fails, but not if no files were found.
"""
try:
for path, (size, updated) in blobstorageio.BlobStorageIO() \
for path, (size, updated) in self._blobstorageIO() \
.list_prefix(dir_or_prefix, with_metadata=True).items():
yield FileMetadata(path, size, updated)
except Exception as e: # pylint: disable=broad-except
raise BeamIOError("List operation failed", {dir_or_prefix: e})

def _blobstorageIO(self):
return blobstorageio.BlobStorageIO(pipeline_options=self._pipeline_options)

def _path_open(
self,
path,
Expand All @@ -134,8 +141,7 @@ def _path_open(
"""
compression_type = FileSystem._get_compression_type(path, compression_type)
mime_type = CompressionTypes.mime_type(compression_type, mime_type)
raw_file = blobstorageio.BlobStorageIO().open(
path, mode, mime_type=mime_type)
raw_file = self._blobstorageIO().open(path, mode, mime_type=mime_type)
if compression_type == CompressionTypes.UNCOMPRESSED:
return raw_file
return CompressedFile(raw_file, compression_type=compression_type)
Expand Down Expand Up @@ -190,7 +196,7 @@ def copy(self, source_file_names, destination_file_names):
message = 'Unable to copy unequal number of sources and destinations.'
raise BeamIOError(message)
src_dest_pairs = list(zip(source_file_names, destination_file_names))
return blobstorageio.BlobStorageIO().copy_paths(src_dest_pairs)
return self._blobstorageIO().copy_paths(src_dest_pairs)

def rename(self, source_file_names, destination_file_names):
"""Rename the files at the source list to the destination list.
Expand All @@ -207,7 +213,7 @@ def rename(self, source_file_names, destination_file_names):
message = 'Unable to rename unequal number of sources and destinations.'
raise BeamIOError(message)
src_dest_pairs = list(zip(source_file_names, destination_file_names))
results = blobstorageio.BlobStorageIO().rename_files(src_dest_pairs)
results = self._blobstorageIO().rename_files(src_dest_pairs)
# Retrieve exceptions.
exceptions = {(src, dest): error
for (src, dest, error) in results if error is not None}
Expand All @@ -223,7 +229,7 @@ def exists(self, path):
Returns: boolean flag indicating if path exists
"""
try:
return blobstorageio.BlobStorageIO().exists(path)
return self._blobstorageIO().exists(path)
except Exception as e: # pylint: disable=broad-except
raise BeamIOError("Exists operation failed", {path: e})

Expand All @@ -239,7 +245,7 @@ def size(self, path):
``BeamIOError``: if path doesn't exist.
"""
try:
return blobstorageio.BlobStorageIO().size(path)
return self._blobstorageIO().size(path)
except Exception as e: # pylint: disable=broad-except
raise BeamIOError("Size operation failed", {path: e})

Expand All @@ -255,7 +261,7 @@ def last_updated(self, path):
``BeamIOError``: if path doesn't exist.
"""
try:
return blobstorageio.BlobStorageIO().last_updated(path)
return self._blobstorageIO().last_updated(path)
except Exception as e: # pylint: disable=broad-except
raise BeamIOError("Last updated operation failed", {path: e})

Expand All @@ -272,7 +278,7 @@ def checksum(self, path):
``BeamIOError``: if path isn't a file or doesn't exist.
"""
try:
return blobstorageio.BlobStorageIO().checksum(path)
return self._blobstorageIO().checksum(path)
except Exception as e: # pylint: disable=broad-except
raise BeamIOError("Checksum operation failed", {path, e})

Expand All @@ -289,7 +295,7 @@ def metadata(self, path):
``BeamIOError``: if path isn't a file or doesn't exist.
"""
try:
file_metadata = blobstorageio.BlobStorageIO()._status(path)
file_metadata = self._blobstorageIO()._status(path)
return FileMetadata(
path, file_metadata['size'], file_metadata['last_updated'])
except Exception as e: # pylint: disable=broad-except
Expand All @@ -305,7 +311,7 @@ def delete(self, paths):
Raises:
``BeamIOError``: if any of the delete operations fail
"""
results = blobstorageio.BlobStorageIO().delete_paths(paths)
results = self._blobstorageIO().delete_paths(paths)
# Retrieve exceptions.
exceptions = {
path: error
Expand Down
24 changes: 12 additions & 12 deletions sdks/python/apache_beam/io/azure/blobstoragefilesystem_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ def test_match_single(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
blobstorageio_mock.exists.return_value = True
blobstorageio_mock._status.return_value = {
'size': 1, 'last_updated': 99999.0
Expand All @@ -107,7 +107,7 @@ def test_match_multiples(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
blobstorageio_mock.list_prefix.return_value = {
'azfs://storageaccount/container/file1': (1, 99999.0),
'azfs://storageaccount/container/file2': (2, 88888.0)
Expand All @@ -128,7 +128,7 @@ def test_match_multiples_limit(self, unused_mock_blobstorageio):
blobstorageio_mock = mock.MagicMock()
limit = 1
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
blobstorageio_mock.list_prefix.return_value = {
'azfs://storageaccount/container/file1': (1, 99999.0)
}
Expand All @@ -146,7 +146,7 @@ def test_match_multiples_error(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
exception = IOError('Failed')
blobstorageio_mock.list_prefix.side_effect = exception

Expand All @@ -165,7 +165,7 @@ def test_match_multiple_patterns(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
blobstorageio_mock.list_prefix.side_effect = [
{
'azfs://storageaccount/container/file1': (1, 99999.0)
Expand All @@ -189,7 +189,7 @@ def test_create(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
# Issue file copy.
_ = self.fs.create(
'azfs://storageaccount/container/file1', 'application/octet-stream')
Expand All @@ -204,7 +204,7 @@ def test_open(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
# Issue file copy.
_ = self.fs.open(
'azfs://storageaccount/container/file1', 'application/octet-stream')
Expand All @@ -219,7 +219,7 @@ def test_copy_file(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
sources = [
'azfs://storageaccount/container/from1',
'azfs://storageaccount/container/from2',
Expand All @@ -240,7 +240,7 @@ def test_copy_file_error(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
sources = [
'azfs://storageaccount/container/from1',
'azfs://storageaccount/container/from2',
Expand All @@ -260,7 +260,7 @@ def test_delete(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
blobstorageio_mock.size.return_value = 0
files = [
'azfs://storageaccount/container/from1',
Expand All @@ -276,7 +276,7 @@ def test_delete_error(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock
nonexistent_directory = 'azfs://storageaccount/nonexistent-container/tree/'
exception = blobstorageio.BlobStorageError('Not found', 404)

Expand Down Expand Up @@ -307,7 +307,7 @@ def test_rename(self, unused_mock_blobstorageio):
# Prepare mocks.
blobstorageio_mock = mock.MagicMock()
blobstoragefilesystem.blobstorageio.BlobStorageIO = \
lambda: blobstorageio_mock
lambda pipeline_options: blobstorageio_mock

sources = [
'azfs://storageaccount/container/original_blob1',
Expand Down
17 changes: 14 additions & 3 deletions sdks/python/apache_beam/io/azure/blobstorageio.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,12 @@
import tempfile
import time

from apache_beam.internal.azure import auth
from apache_beam.io.filesystemio import Downloader
from apache_beam.io.filesystemio import DownloaderStream
from apache_beam.io.filesystemio import Uploader
from apache_beam.io.filesystemio import UploaderStream
from apache_beam.options.pipeline_options import AzureOptions
from apache_beam.utils import retry

_LOGGER = logging.getLogger(__name__)
Expand Down Expand Up @@ -105,10 +107,19 @@ def __init__(self, message=None, code=None):

class BlobStorageIO(object):
"""Azure Blob Storage I/O client."""
def __init__(self, client=None):
connect_str = os.getenv('AZURE_STORAGE_CONNECTION_STRING')
def __init__(self, client=None, pipeline_options=None):
if client is None:
self.client = BlobServiceClient.from_connection_string(connect_str)
azure_options = pipeline_options.view_as(AzureOptions)
connect_str = azure_options.azure_connection_string or \
os.getenv('AZURE_STORAGE_CONNECTION_STRING')
if connect_str:
self.client = BlobServiceClient.from_connection_string(
conn_str=connect_str)
else:
credential = auth.get_service_credentials(pipeline_options)
self.client = BlobServiceClient(
account_url=azure_options.blob_service_endpoint,
credential=credential)
else:
self.client = client
if not AZURE_DEPS_INSTALLED:
Expand Down
Loading