Skip to content

Commit

Permalink
Adds an s3 list prefixes operator (#17145)
Browse files Browse the repository at this point in the history
- Adds an operator to return a list of prefixes from an S3 bucket
- Updates `list_prefixes()` unit test to assert on a nested dir with a prefix variable
- Removes duplicate calls to `list_keys()` that were in the `test_list_prefixes()` unit test (likely a copy/paste boo boo?)

There are two suggestion from [this conversation](#8464) that I have not included here:
1. Combine or otherwise simplify `s3_list_keys()` and `s3_list_prefixes()` into one - this makes sense to me but I don't quite know how people tend to use these operators or if there is a valid argument for keeping them separate. 

2. Combining all the s3 operators into one file like [gcs.py](https://github.com/apache/airflow/blob/main/airflow/providers/google/cloud/operators/gcs.py) - this also makes sense to me, but it's not consistent with the other AWS operators. Might be worth opening a new issue to refactor them all if we want to go in this direction?

Issue Link: #[8448](#8448)
  • Loading branch information
jarfgit authored Oct 5, 2021
1 parent 1697617 commit 760bf6e
Show file tree
Hide file tree
Showing 4 changed files with 147 additions and 2 deletions.
97 changes: 97 additions & 0 deletions airflow/providers/amazon/aws/operators/s3_list_prefixes.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
#
# 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 typing import Iterable, Optional, Union

from airflow.models import BaseOperator
from airflow.providers.amazon.aws.hooks.s3 import S3Hook


class S3ListPrefixesOperator(BaseOperator):
"""
List all subfolders from the bucket with the given string prefix in name.
This operator returns a python list with the name of all subfolders which
can be used by `xcom` in the downstream task.
:param bucket: The S3 bucket where to find the subfolders. (templated)
:type bucket: str
:param prefix: Prefix string to filter the subfolders whose name begin with
such prefix. (templated)
:type prefix: str
:param delimiter: the delimiter marks subfolder hierarchy. (templated)
:type delimiter: str
:param aws_conn_id: The connection ID to use when connecting to S3 storage.
:type aws_conn_id: str
:param verify: Whether or not to verify SSL certificates for S3 connection.
By default SSL certificates are verified.
You can provide the following values:
- ``False``: do not validate SSL certificates. SSL will still be used
(unless use_ssl is False), but SSL certificates will not be
verified.
- ``path/to/cert/bundle.pem``: A filename of the CA cert bundle to uses.
You can specify this argument if you want to use a different
CA cert bundle than the one used by botocore.
:type verify: bool or str
**Example**:
The following operator would list all the subfolders
from the S3 ``customers/2018/04/`` prefix in the ``data`` bucket. ::
s3_file = S3ListPrefixesOperator(
task_id='list_s3_prefixes',
bucket='data',
prefix='customers/2018/04/',
delimiter='/',
aws_conn_id='aws_customers_conn'
)
"""

template_fields: Iterable[str] = ('bucket', 'prefix', 'delimiter')
ui_color = '#ffd700'

def __init__(
self,
*,
bucket: str,
prefix: str,
delimiter: str,
aws_conn_id: str = 'aws_default',
verify: Optional[Union[str, bool]] = None,
**kwargs,
):
super().__init__(**kwargs)
self.bucket = bucket
self.prefix = prefix
self.delimiter = delimiter
self.aws_conn_id = aws_conn_id
self.verify = verify

def execute(self, context):
hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify)

self.log.info(
'Getting the list of subfolders from bucket: %s in prefix: %s (Delimiter %s)',
self.bucket,
self.prefix,
self.delimiter,
)

return hook.list_prefixes(bucket_name=self.bucket, prefix=self.prefix, delimiter=self.delimiter)
1 change: 1 addition & 0 deletions airflow/providers/amazon/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,7 @@ operators:
- airflow.providers.amazon.aws.operators.s3_delete_objects
- airflow.providers.amazon.aws.operators.s3_file_transform
- airflow.providers.amazon.aws.operators.s3_list
- airflow.providers.amazon.aws.operators.s3_list_prefixes
- integration-name: Amazon SageMaker
python-modules:
- airflow.providers.amazon.aws.operators.sagemaker_base
Expand Down
7 changes: 5 additions & 2 deletions tests/providers/amazon/aws/hooks/test_s3.py
Original file line number Diff line number Diff line change
Expand Up @@ -124,11 +124,14 @@ def test_list_prefixes(self, s3_bucket):
bucket = hook.get_bucket(s3_bucket)
bucket.put_object(Key='a', Body=b'a')
bucket.put_object(Key='dir/b', Body=b'b')
bucket.put_object(Key='dir/sub_dir/c', Body=b'c')

assert [] == hook.list_prefixes(s3_bucket, prefix='non-existent/')
assert [] == hook.list_prefixes(s3_bucket)
assert ['dir/'] == hook.list_prefixes(s3_bucket, delimiter='/')
assert ['a'] == hook.list_keys(s3_bucket, delimiter='/')
assert ['dir/b'] == hook.list_keys(s3_bucket, prefix='dir/')
assert [] == hook.list_prefixes(s3_bucket, prefix='dir/')
assert ['dir/sub_dir/'] == hook.list_prefixes(s3_bucket, delimiter='/', prefix='dir/')
assert [] == hook.list_prefixes(s3_bucket, prefix='dir/sub_dir/')

def test_list_prefixes_paged(self, s3_bucket):
hook = S3Hook()
Expand Down
44 changes: 44 additions & 0 deletions tests/providers/amazon/aws/operators/test_s3_list_prefixes.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
#
# 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.

import unittest
from unittest import mock

from airflow.providers.amazon.aws.operators.s3_list_prefixes import S3ListPrefixesOperator

TASK_ID = 'test-s3-list-prefixes-operator'
BUCKET = 'test-bucket'
DELIMITER = '/'
PREFIX = 'test/'
MOCK_SUBFOLDERS = ['test/']


class TestS3ListOperator(unittest.TestCase):
@mock.patch('airflow.providers.amazon.aws.operators.s3_list_prefixes.S3Hook')
def test_execute(self, mock_hook):

mock_hook.return_value.list_prefixes.return_value = MOCK_SUBFOLDERS

operator = S3ListPrefixesOperator(task_id=TASK_ID, bucket=BUCKET, prefix=PREFIX, delimiter=DELIMITER)

subfolders = operator.execute(None)

mock_hook.return_value.list_prefixes.assert_called_once_with(
bucket_name=BUCKET, prefix=PREFIX, delimiter=DELIMITER
)
assert subfolders == MOCK_SUBFOLDERS

0 comments on commit 760bf6e

Please sign in to comment.