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 2021/09/03 03:21:15 UTC

[GitHub] [airflow] jarfgit opened a new pull request #17145: Adds an s3 list prefixes operator

jarfgit opened a new pull request #17145:
URL: https://github.com/apache/airflow/pull/17145


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


-- 
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 change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r722432259



##########
File path: airflow/providers/amazon/aws/operators/s3_list.py
##########
@@ -72,8 +72,8 @@ def __init__(
         self,
         *,
         bucket: str,
-        prefix: str = '',
-        delimiter: str = '',
+        prefix: str,
+        delimiter: str,

Review comment:
       Is this change required?




-- 
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 #17145: Adds an s3 list prefixes operator

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


   


-- 
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] o-nikolas commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-898595938


   > * The user can distinguish keys from prefixes from the list returned by the operator. I would assume that keys have a file extension and prefixes would include the delimiter... but it seems possible that keys may not _always_ have a file extension and I'm basing my understanding of the prefixes containing the delimiter on the unit tests.
   
   This doesn't sound overly confident πŸ˜†
   
   > 
   > So I have this question:
   > 
   > If we want to refactor the operator to return both prefixes and keys but a user might want to use different optional params between keys and prefixes, I don't see an alternative other than requiring the user to use the operator twice with different params. With this in mind, is there a valid argument to have a dedicated `S3ListPrefixes` operator after all?
   
   It does sound like you're finding significant friction here. Two paths forward I see are:
   1. Change the type of the new parameter to something more advanced that will allow the user to specify exactly what they want out of the command (something like a string or an enum, that can be `keys`, `prefixes`, or `all`). This doesn't get around the issue of having two calls for different optional params though. But, at that point, the user really should have two separate calls if they are querying for a completely distinct set of results IMHO.
   1. Or, just bail on a single operator and go back to having two as you suggest.
   
   What do others 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] boring-cyborg[bot] commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-884414675


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better πŸš€.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
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] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r720563705



##########
File path: tests/providers/amazon/aws/operators/test_s3_list_prefixes.py
##########
@@ -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 sorted(subfolders) == sorted(MOCK_SUBFOLDERS)

Review comment:
       We do not, updated in the latest push.




-- 
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] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r722435325



##########
File path: airflow/providers/amazon/aws/operators/s3_list.py
##########
@@ -72,8 +72,8 @@ def __init__(
         self,
         *,
         bucket: str,
-        prefix: str = '',
-        delimiter: str = '',
+        prefix: str,
+        delimiter: str,

Review comment:
       Actually no. I copied this code for the S3ListPrefixes operator - where you'd presumably want these to be explicit values, but that doesn't necessarily apply to S3List. Let me revert this and push it up. Good catch :)




-- 
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] o-nikolas commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-918681350


   > @o-nikolas I went ahead and reverted back to having two distinct operators - if for no other reason than to keep this PR / convo alive. If I had a better understanding of s3 list keys / prefixes the use cases I would feel more confident exploring the first path you outlined above.
   > 
   > That being said, still open to workshopping this :)
   
   Sorry I've been on vacation for the past three weeks:
   
   Sounds good to me, I think that's perfectly valid :) I'll try have a review sometime this week.


-- 
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] jarfgit edited a comment on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit edited a comment on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-893890145


   @o-nikolas @iostreamdoth @potiuk Ok, at long last I've updated this pull request:
   
   * Refactored `S3ListOperator` to take a `recursive` parameter
   * Refactored the `s3_list_keys` hook to return both a list of keys and list of common prefixes (I know this is out of scope for the issue, but making unnecessary API calls bothers me and it was easy enough to do)
   * Experimented with deleting the `s3_list_prefixes` hook altogether, but it's used elsewhere and constituted some serious scope creep πŸ˜› 
   * If `recursive == True` then we add the prefixes to the list of keys returned by the operator
   * Ensured `recursive` defaults to `False` and _shouldn't_ present a breaking change
   * Added unit tests to both the `s3_list_keys` and `s3_list_prefixes` hooks to make sure I (and presumably future contributors) fully understood what the `delimiter` and `prefix` args were doing and how various combinations affected what was returned when `recursive == True`
   
   
   The above assumes the following:
   
   * The user wants to retrieve both keys and prefixes using the same optional params (i.e. `delimiter` and `prefix`). 
   * The user can distinguish keys from prefixes from the list returned by the operator. I would assume that keys have a file extension and prefixes would include the delimiter... but it seems possible that keys may not _always_ have a file extension and I'm basing my understanding of the prefixes containing the delimiter on the unit tests.
   
   
   So I have this question:
   
   If we want to refactor the operator to return both prefixes and keys but a user might want to use different optional params between keys and prefixes, I don't see an alternative other than requiring the user to use the operator twice with different params. With this in mind, is there a valid argument to have a dedicated `S3ListPrefixes` operator after all?
   
   


-- 
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 change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r722440711



##########
File path: airflow/providers/amazon/aws/operators/s3_list.py
##########
@@ -72,8 +72,8 @@ def __init__(
         self,
         *,
         bucket: str,
-        prefix: str = '',
-        delimiter: str = '',
+        prefix: str,
+        delimiter: str,

Review comment:
       ```suggestion
           prefix: str = '',
           delimiter: str = '',
   ```




-- 
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] iostreamdoth commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
iostreamdoth commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-886827072


   I propose if we can rename parameter to `recursive` in place of (`include_subfolders or `include_common_prefixes`) in `s3listoperator` to be in concordance with `aws s3 ls` command line tool options. Default recursive is true to retain the current behavior of operator.
   thought @o-nikolas @jarfgit  


-- 
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] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r701227810



##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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,
+        )

Review comment:
       Oops :)




-- 
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] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r701226027



##########
File path: tests/providers/amazon/aws/hooks/test_s3.py
##########
@@ -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/')

Review comment:
       The calls to `list_keys` were in the `test_list_prefixes` unit test. I didn't immediately see why these calls were there and assumed it might have been a copy / past error. Am I missing something?




-- 
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] jarfgit commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-886850297


   @potiuk still working on it. I'm actually on the airflow team at astronomer and got pulled into something else. I'll circle back on this shortly. Sorry for the delay!


-- 
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] jarfgit commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-910804801


   @o-nikolas I went ahead and reverted back to having two distinct operators - if for no other reason than to keep this PR / convo alive. If I had a better understanding of s3 list keys / prefixes the use cases I would feel more confident exploring the first path you outlined above.
   
   That being said, still open to workshopping this :)


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

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

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



[GitHub] [airflow] uranusjr closed pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
uranusjr closed pull request #17145:
URL: https://github.com/apache/airflow/pull/17145


   


-- 
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] uranusjr commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r700782607



##########
File path: tests/providers/amazon/aws/hooks/test_s3.py
##########
@@ -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/')

Review comment:
       Are these changes related to the operator addition?




-- 
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] o-nikolas commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-886868621


   > I propose if we can rename parameter to `recursive` in place of (`include_subfolders` or `include_common_prefixes`) in `s3listoperator` to be in concordance with `aws s3 ls` command line tool options. Default recursive is true to retain the current behavior of operator.
   > thoughts @o-nikolas @jarfgit ?
   
   I think that's a perfectly fine name, but I don't like the reasoning of tying it to the `aws s3 ls` command. If anything I think we should be tying to choose naming that closely follows the boto api that this operator wraps (`list_objects_v2`) and that api uses the common S3 language of `prefixes`. But again, I think `recursive` is expressive enough and I'm happy to agree to that :) 


-- 
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] potiuk commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-886234926


   Hey @jarfgit  - any news :) ? 


-- 
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] jarfgit commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-893890145


   @o-nikolas @iostreamdoth @potiuk Ok, at long last I've updated this pull request:
   
   * Refactored `S3ListOperator` to take a `recursive` parameter
   * Refactored the S3 list keys hook to return both a list of keys and list of common prefixes (I know this is out of scope for the issue, but making unnecessary API calls bothers me and it was easy enough to do)
   * If `recursive == True` then we add the prefixes to the list of keys returned by the operator
   * Ensured `recursive` defaults to `False` and _shouldn't_ present a breaking change
   * Added unit tests to both the `s3_list_keys` and `s3_list_prefixes` hooks to make sure I (and presumably future contributors) fully understood what the `delimiter` and `prefix` args were doing and how various combinations affected what was returned when `recursive == True`
   
   
   The above assumes the following:
   
   * The user wants to retrieve both keys and prefixes using the same optional params (i.e. `delimiter` and `prefix`). 
   * The user can distinguish keys from prefixes from the list returned by the operator. I would assume that keys have a file extension and prefixes would include the delimiter... but it seems possible that keys may not _always_ have a file extension and I'm basing my understanding of the prefixes containing the delimiter on the unit tests.
   
   
   So I have this question:
   
   If we want to refactor the operator to return both prefixes and keys but a user might want to use different optional params between keys and prefixes, I don't see an alternative other than requiring the user to use the operator twice with different params. With this in mind, is there a valid argument to have a dedicated `S3ListPrefixes` operator after all?
   
   


-- 
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] iostreamdoth edited a comment on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
iostreamdoth edited a comment on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-886827072






-- 
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] jarfgit commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-884506028


   @o-nikolas I like what you're thinking. Thanks for giving me more context. I'll see what I can do.


-- 
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] uranusjr commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r700783078



##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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,
+        )

Review comment:
       ```suggestion
           self.log.info(
               'Getting the list of subfolders from bucket: %s in prefix: %s (Delimiter %s)',
               self.bucket,
               self.prefix,
               self.delimiter,
           )
   ```
   
   Typo? πŸ™‚ 




-- 
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] o-nikolas commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r709474340



##########
File path: tests/providers/amazon/aws/operators/test_s3_list_prefixes.py
##########
@@ -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 sorted(subfolders) == sorted(MOCK_SUBFOLDERS)

Review comment:
       Do we need the sort if there is only one item in `MOCK_SUBFOLDERS`?

##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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 = '',

Review comment:
       Does it really make sense to allow the user to not provide these values?




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

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

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



[GitHub] [airflow] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r720568380



##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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 = '',

Review comment:
       @o-nikolas github is not letting me re-request a review from you for some reason, so pinging here.




-- 
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] github-actions[bot] commented on pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#issuecomment-932874333


   The PR is likely OK to be merged with just subset of tests for default Python and Database versions without running the full matrix of tests, because it does not modify the core of Airflow. If the committers decide that the full tests matrix is needed, they will add the label 'full tests needed'. Then you should rebase to the latest main or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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] jarfgit commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
jarfgit commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r720563472



##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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 = '',

Review comment:
       It doesn't :)
   
   I'll admit this was a lazy copy / paste from S3List, so I updated the `prefix` and `delimiter` there as well.




-- 
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] o-nikolas commented on a change in pull request #17145: Adds an s3 list prefixes operator

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on a change in pull request #17145:
URL: https://github.com/apache/airflow/pull/17145#discussion_r722524941



##########
File path: airflow/providers/amazon/aws/operators/s3_list_prefixes.py
##########
@@ -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 = '',

Review comment:
       Hey, sorry I missed this message. Code looks good, and I see that it's merged. Congrats :) 




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