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/02/11 14:31:19 UTC

[GitHub] [airflow] Scuall1992 opened a new pull request #14191: Add new operator gdrive to local

Scuall1992 opened a new pull request #14191:
URL: https://github.com/apache/airflow/pull/14191


   <!--
   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/master/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/master/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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/suite/hooks/drive.py
##########
@@ -218,3 +218,9 @@ def upload_file(self, local_location: str, remote_location: str) -> str:
         )
         self.log.info("File %s uploaded to gdrive://%s.", local_location, remote_location)
         return file.get("id")
+
+    def download_file(self, file_id: str, file_handle):

Review comment:
       A doctoring is required for every public function, otherwise pylons will fail on CI. I recommend installing pre-commits to check it locally:
   https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,96 @@
+# 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 Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def execute(self, context):
+        self.log.info('Executing download: %s into %s', self.filename, self.output_file)
+        gdrive_hook = GoogleDriveHook(
+            delegate_to=self.delegate_to,
+            impersonation_chain=self.impersonation_chain,
+        )
+        self.file_metadata = gdrive_hook.get_file_id(
+            folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+        )
+
+        with open(self.output_file, "wb") as file:
+            gdrive_hook.download_file(file_id=self.file_metadata["id"], file_handle=file)

Review comment:
       ```suggestion
   
   
       def execute(self, context):
           self.log.info('Executing download: %s into %s', self.filename, self.output_file)
           gdrive_hook = GoogleDriveHook(
               delegate_to=self.delegate_to,
               impersonation_chain=self.impersonation_chain,
           )
           file_metadata = gdrive_hook.get_file_id(
               folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
           )
   
           with open(self.output_file, "wb") as file:
               gdrive_hook.download_file(file_id=file_metadata["id"], file_handle=file)
   ```
   Probably no need to keep this as instance attribute




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/suite/hooks/drive.py
##########
@@ -218,3 +218,15 @@ def upload_file(self, local_location: str, remote_location: str) -> str:
         )
         self.log.info("File %s uploaded to gdrive://%s.", local_location, remote_location)
         return file.get("id")
+
+    def download_file(self, file_id: str, file_handle, chunk_size=104857600):

Review comment:
       ```suggestion
       def download_file(self, file_id: str, file_handle: io.TextIOWrapper, chunk_size: int = 104857600):
   ```




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   
   @turbaszek I think it's will be great. Can you explain a problem more deeply? and what I should 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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #14191: Add new operator gdrive to local

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/568756160) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata
+
+    def _download_data(self, gdrive_hook: GoogleDriveHook, output_file: str):
+        file_handle = BytesIO()
+        self._set_file_metadata(gdrive_hook=gdrive_hook)
+        file_id = self.file_metadata["id"]
+        request = gdrive_hook.get_media_request(file_id=file_id)
+        gdrive_hook.download_content_from_request(
+            file_handle=file_handle, request=request, chunk_size=104857600
+        )
+
+        with open(output_file, "wb") as f:
+            f.write(file_handle.getbuffer())

Review comment:
       I think it can make sense to move this method to `GoogleDriveHook` so it can be reused in other operators in future.




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata
+
+    def _download_data(self, gdrive_hook: GoogleDriveHook, output_file: str):
+        file_handle = BytesIO()
+        self._set_file_metadata(gdrive_hook=gdrive_hook)
+        file_id = self.file_metadata["id"]
+        request = gdrive_hook.get_media_request(file_id=file_id)
+        gdrive_hook.download_content_from_request(
+            file_handle=file_handle, request=request, chunk_size=104857600
+        )
+
+        with open(output_file, "wb") as f:
+            f.write(file_handle.getbuffer())

Review comment:
       I think it can make sense to move this method to `GoogleDriveHook` so it can be reused in other operators in future. I would suggest the following:
   
   ```py
   def download_file(self, file_id: str, file_handle):
            request = gdrive_hook.get_media_request(file_id=file_id)
            gdrive_hook.download_content_from_request(
                file_handle=file_handle, request=request, chunk_size=104857600
            )
   ```
   
   and then you can simply do:
   ```py
   with open(output_file, "wb" as file:
       gcs_hook.download_file(file_id=self.file_id, file_handle=file)
   ```




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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`

Review comment:
       Do you have a guide how to add a guide?)))




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   So, what about rebase?
   
   > If that's all, then I should do this commands?
   > 
   > git fetch --all
   > git merge-base my-branch apache/master
   > git checkout my-branch
   > git rebase HASH --onto apache/master
   > git push --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.

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



[GitHub] [airflow] Scuall1992 edited a comment on pull request #14191: Add new operator gdrive to local

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


   Hello, for the test of operator I should just check that gdrive hook is called?


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

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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #14191: Add new operator gdrive to local

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


   Awesome work, congrats on your first merged pull request!
   


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

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



[GitHub] [airflow] turbaszek edited a comment on pull request #14191: Add new operator gdrive to local

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


   @Scuall1992 amazing work! Would you like to consider working on `GdriveToGCSOperator`?  All the logic is already there so it should be pretty straightforward if we use:
   https://github.com/apache/airflow/blob/e4629b6211bdd41340b113cda4925318b5655386/airflow/providers/google/cloud/hooks/gcs.py#L335-L343


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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   > @turbaszek does it really make sense for airflow to have _ToLocal operators?
   > 
   > Seems like an odd use case? Is the idea to use it with a mounted volume or something?
   
   @dstandish we already have few `Local` operators and they use case is very valid for single machine deployments using `LocalExecutor`. It would be good to add something like `GDriveToGCSOperator` but I think this can be done as a follow up - we will just simply reuse hooks method. 


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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
##########
@@ -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.
+
+
+Downloads data from Google Drive Storage to Local Filesystem
+============================================================
+The `Google Drive <https://www.google.com/drive/>`__ is
+used to store daily use data, including documents and photos. Google Drive has built-in mechanisms to facilitate group work e.g.
+document editor, file sharing mechanisms.
+
+.. contents::
+  :depth: 1
+  :local:
+
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+.. _howto/operator:GoogleDriveToLocalOperator:
+
+GCSToLocalFilesystemOperator
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+:class:`~airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveToLocalOperator` allows you to download
+data from Google Drive to local filesystem.
+
+
+Below is an example of using this operator to download file from Google Drive to Local Filesystem.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py
+    :language: python
+    :dedent: 0

Review comment:
       What does it mean? dedent
   




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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata

Review comment:
       so maybe I'll wipe _set_file_metadata function and will use get_file_id directly? I didn't found another usages of this function




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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   @Scuall1992 the idea is to have operator similar to `GoogleDriveToLocalOperator` which instead of downloading file to local file system will transfer this file to Google Cloud Storage. It should do something similar to this in `execcute`:
   ```py
   with gdrive_hook.provide_file_and_upload(...) as file:
       gdrive_hook.download_file(file_handle=file, ...)
   ```
   Both of those methods are already implemented πŸ‘Œ 


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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   I did everything you noticed. Pre-commit passed every checks.
   
   If that's all, then I should do this commands?
   
   git fetch --all 
   git merge-base my-branch apache/master 
   git checkout my-branch 
   git rebase HASH --onto apache/master 
   git push --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.

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: tests/providers/google/cloud/transfers/test_gdrive_to_local.py
##########
@@ -0,0 +1,38 @@
+#
+# 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 unittest import TestCase, mock
+
+from airflow.providers.google.cloud.transfers.gdrive_to_local import GoogleDriveToLocalOperator
+
+TASK_ID = "test-drive-to-local-operator"
+FOLDER_ID = "1234567890qwerty"
+FILE_NAME = "file.pdf"
+OUTPUT_FILE = "out_file.pdf"
+
+
+class TestGoogleDriveToLocalOperator(TestCase):
+    @mock.patch("airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveHook")
+    def test_execute(self, hook_mock):
+        op = GoogleDriveToLocalOperator(
+            task_id=TASK_ID,
+            folder_id=FOLDER_ID,
+            file_name=FILE_NAME,
+            output_file=OUTPUT_FILE,
+        )
+        op.execute(context=None)
+        hook_mock.assert_called_once()

Review comment:
       But how can I mock file_metadata?
   




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,95 @@
+# 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 Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`

Review comment:
       ```suggestion
           :ref:`howto/operator: GoogleDriveToLocalOperator `
   ```




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   @turbaszek  thank you very much for your help.
   
   Can you please check did I made rebase correctly?
   
   I ran your commands 
   
   git fetch --all
   git rebase apache/master
   git push --force-with-lease
   
   From my forked project, where I pushed pull request, from branch feature-google-download-branch


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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   > Hello, for the test of operator I should just check that gdrive hook is called?
   
   Yes, that would be enough


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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   @Scuall1992 amazing work! Would like to consider working on `GdriveToGCSOperator`?  All the logic is already there so it should be pretty straightforward if we use:
   https://github.com/apache/airflow/blob/e4629b6211bdd41340b113cda4925318b5655386/airflow/providers/google/cloud/hooks/gcs.py#L335-L343


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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   > So, what about rebase?
   
   I'm not sure if it's needed right now - everything looks good. But for future reference I usually do:
   ```
   git fetch --all
   git rebase apache/master
   git push --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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`

Review comment:
       ```suggestion
           :ref:`howto/operator: GoogleDriveToLocalOperator `
   ```
   This will require a file in `docs/apache-airflow-providers-google/operators/suite/` with how-to guid πŸ‘Œ 




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/suite/hooks/drive.py
##########
@@ -218,3 +218,9 @@ def upload_file(self, local_location: str, remote_location: str) -> str:
         )
         self.log.info("File %s uploaded to gdrive://%s.", local_location, remote_location)
         return file.get("id")
+
+    def download_file(self, file_id: str, file_handle):

Review comment:
       A docstring is required for every public function, otherwise pylint will fail on CI. I recommend installing pre-commits to check it locally:
   https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,95 @@
+# 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 Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):

Review comment:
       Could you please add unit test for this operator? This examples should help you:
   https://github.com/apache/airflow/blob/cb4a60e9d059eeeae02909bb56a348272a55c233/tests/providers/google/marketing_platform/operators/test_search_ads.py#L31-L64
   




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #14191: Add new operator gdrive to local

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


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: tests/providers/google/cloud/transfers/test_gdrive_to_local.py
##########
@@ -0,0 +1,38 @@
+#
+# 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 unittest import TestCase, mock
+
+from airflow.providers.google.cloud.transfers.gdrive_to_local import GoogleDriveToLocalOperator
+
+TASK_ID = "test-drive-to-local-operator"
+FOLDER_ID = "1234567890qwerty"
+FILE_NAME = "file.pdf"
+OUTPUT_FILE = "out_file.pdf"
+
+
+class TestGoogleDriveToLocalOperator(TestCase):
+    @mock.patch("airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveHook")
+    def test_execute(self, hook_mock):
+        op = GoogleDriveToLocalOperator(
+            task_id=TASK_ID,
+            folder_id=FOLDER_ID,
+            file_name=FILE_NAME,
+            output_file=OUTPUT_FILE,
+        )
+        op.execute(context=None)
+        hook_mock.assert_called_once()

Review comment:
       ```suggestion
           hook_mock.assert_called_once_with(delegate_to=None, impersonation_chain=None)
   ```
   This only verify instantiating the hook. We also need to check if right method was called so we need:
   ```py
   hook_mock.return_value.get_file_id.assert_called_once_with(...)
   hook_mock.return_value.download_file.assert_called_once_with(...)
   ```
   where the `...` needs to be replaced with proper values. Also consider using
   ```py
   with TemporaryNamedFile("wb") as temp_file:
       op = GoogleDriveToLocalOperator(..., output_file=temp_file.name)
   ```
   to avoid a side effect in form of a created file.




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

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



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #14191: Add new operator gdrive to local

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


   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/master/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, pylint and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/master/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/master/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/master/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/master/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.

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   Hello, can anybody explain the difference and purposes of test with and without 'system' prefix? 
   And for the test of operator I should just check that gdrive hook is called?


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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   I have added rst file and example. Can you please check am I doing right?


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/suite/hooks/drive.py
##########
@@ -218,3 +218,15 @@ def upload_file(self, local_location: str, remote_location: str) -> str:
         )
         self.log.info("File %s uploaded to gdrive://%s.", local_location, remote_location)
         return file.get("id")
+
+    def download_file(self, file_id: str, file_handle):
+        """
+        Download a file from Google Drive.
+
+        :param file_id: the id of the file
+        :type file_id: str
+        :param file_handle: file handle used to write the content to
+        :type file_handle: io.TextIOWrapper
+        """
+        request = self.get_media_request(file_id=file_id)
+        self.download_content_from_request(file_handle=file_handle, request=request, chunk_size=104857600)

Review comment:
       Should we make the `chunk_size` configurable? 




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

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



[GitHub] [airflow] RNHTTR commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`

Review comment:
       There are a bunch of examples in `docs/apache-airflow-providers-google/operators`. 
   
   This was a challenge for me as well. It might be worth it to have a doc similar to [the Contribution Quickstart Guide](https://github.com/apache/airflow/blob/master/CONTRIBUTORS_QUICK_START.rst) but specifically for contributing to google components?




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata

Review comment:
       Personally I would just call `gdrive_hook.get_file_id` in the execute method. The `if not self.file_metadata` will always be true in execution context. 




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata
+
+    def _download_data(self, gdrive_hook: GoogleDriveHook, output_file: str):
+        file_handle = BytesIO()
+        self._set_file_metadata(gdrive_hook=gdrive_hook)
+        file_id = self.file_metadata["id"]
+        request = gdrive_hook.get_media_request(file_id=file_id)
+        gdrive_hook.download_content_from_request(
+            file_handle=file_handle, request=request, chunk_size=104857600
+        )
+
+        with open(output_file, "wb") as f:
+            f.write(file_handle.getbuffer())

Review comment:
       I think it can make sense to move this method to `GoogleDriveHook` so it can be reused in other operators in future. I would suggest the following:
   
   ```py
   def download_file(self, file_id: str, file_handle):
            request = gdrive_hook.get_media_request(file_id=file_id)
            gdrive_hook.download_content_from_request(
                file_handle=file_handle, request=request, chunk_size=104857600
            )
   ```
   
   and then you can simply do in `execute` method:
   ```py
   with open(output_file, "wb") as file:
       gcs_hook.download_file(file_id=self.file_id, file_handle=file)
   ```




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   @turbaszek  what the difference with this file https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py
   ?
   


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,96 @@
+# 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 Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,

Review comment:
       ```suggestion
           file_name: str,
   ```
   Let's rename this parameter to be consistent with what we have in existing operators. Also this should fix the CI error (dag uses `file_name` but your operator requires `filename`). I recommend using pre-commit, it helps catching such issues πŸ‘Œ 
   




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

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



[GitHub] [airflow] dstandish commented on pull request #14191: Add new operator gdrive to local

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


   @turbaszek does it really make sense for airflow to have _ToLocal operators?
   
   Seems like an odd use case?   Is the idea to use it with a mounted volume or something?
   
   Commonly, airflow workers are ephemeral and nothing local is persisted....  and what's local on one task might not present on another...


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
##########
@@ -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.
+
+
+Downloads data from Google Drive Storage to Local Filesystem
+============================================================
+The `Google Drive <https://www.google.com/drive/>`__ is
+used to store daily use data, including documents and photos. Google Drive has built-in mechanisms to facilitate group work e.g.
+document editor, file sharing mechanisms.
+
+.. contents::
+  :depth: 1
+  :local:
+
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+.. _howto/operator:GoogleDriveToLocalOperator:
+
+GCSToLocalFilesystemOperator
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+:class:`~airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveToLocalOperator` allows you to download
+data from Google Drive to local filesystem.
+
+
+Below is an example of using this operator to download file from Google Drive to Local Filesystem.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py
+    :language: python
+    :dedent: 0

Review comment:
       It will remove indentation from the example (remove leading spaces). So instead of this:
   ```py
       op = MyOperator(...)
   ```
   documentation will include
   ```py
   op = MyOperator(...)
   ```




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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   > How can I fix this?
   > Items in the first set but not the second:
   > '/docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst'
   
   Add this new operator to this list:
   https://github.com/apache/airflow/blob/e3bcaa3ba351234effe52ad380345c4e39003fcb/airflow/providers/google/provider.yaml#L428-L430
   
   All your work looks good! πŸš€ 
   


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

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



[GitHub] [airflow] turbaszek merged pull request #14191: Add new operator gdrive to local

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


   


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
##########
@@ -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.
+
+
+Downloads data from Google Drive Storage to Local Filesystem
+============================================================
+The `Google Drive <https://www.google.com/drive/>`__ is
+used to store daily use data, including documents and photos. Google Drive has built-in mechanisms to facilitate group work e.g.
+document editor, file sharing mechanisms.
+
+.. contents::
+  :depth: 1
+  :local:
+
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+.. _howto/operator:GoogleDriveToLocalOperator:
+
+GCSToLocalFilesystemOperator
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+:class:`~airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveToLocalOperator` allows you to download
+data from Google Drive to local filesystem.
+
+
+Below is an example of using this operator to download file from Google Drive to Local Filesystem.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gdrive_download_file_task]
+    :end-before: [END howto_operator_gdrive_download_file_task]

Review comment:
       ```suggestion
       :start-after: [START download_from_gdrive_to_local]
       :end-before: [END download_from_gdrive_to_local]
   ```
   This has to be aligned with what we have in the example DAG.




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/suite/hooks/drive.py
##########
@@ -218,3 +218,9 @@ def upload_file(self, local_location: str, remote_location: str) -> str:
         )
         self.log.info("File %s uploaded to gdrive://%s.", local_location, remote_location)
         return file.get("id")
+
+    def download_file(self, file_id: str, file_handle):

Review comment:
       ```suggestion
       def download_file(self, file_id: str, file_handle):
           """
           Download a file from Google Drive.
           
           :param file_id: the id of the file
           :type file_id: str
           :param file_handle: file handle used to write the content to
           :type file_handle: io.TextIOWrapper
           """
   ```
   A docstring is required for every public function, otherwise pylint will fail on CI. I recommend installing pre-commits to check it locally:
   https://github.com/apache/airflow/blob/master/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
##########
@@ -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.
+
+
+Downloads data from Google Drive Storage to Local Filesystem
+============================================================
+The `Google Drive <https://www.google.com/drive/>`__ is
+used to store daily use data, including documents and photos. Google Drive has built-in mechanisms to facilitate group work e.g.
+document editor, file sharing mechanisms.
+
+.. contents::
+  :depth: 1
+  :local:
+
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+.. _howto/operator:GoogleDriveToLocalOperator:
+
+GCSToLocalFilesystemOperator
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+:class:`~airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveToLocalOperator` allows you to download
+data from Google Drive to local filesystem.
+
+
+Below is an example of using this operator to download file from Google Drive to Local Filesystem.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py
+    :language: python
+    :dedent: 0

Review comment:
       ```suggestion
       :dedent: 4
   ```




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   Okay, then I'm waiting till the end of the CI build checks and then run this commands
   


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #14191: Add new operator gdrive to local

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/561950737) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] dstandish edited a comment on pull request #14191: Add new operator gdrive to local

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


   @turbaszek does it really make sense for airflow to have _ToLocal operators?
   
   Seems like an odd use case?   Is the idea to use it with a mounted volume or something?
   
   Commonly, airflow workers are ephemeral and nothing local is persisted....  and what's local on one task might not present on another...
   
   Maybe it's better to just have a hook method that downloads to local, and user who need this kind of thing can easily wrap it into and operator themselves.  That would be my suggestion i guess.


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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
##########
@@ -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.
+
+
+Downloads data from Google Drive Storage to Local Filesystem
+============================================================
+The `Google Drive <https://www.google.com/drive/>`__ is
+used to store daily use data, including documents and photos. Google Drive has built-in mechanisms to facilitate group work e.g.
+document editor, file sharing mechanisms.
+
+.. contents::
+  :depth: 1
+  :local:
+
+
+Prerequisite Tasks
+^^^^^^^^^^^^^^^^^^
+
+.. include::/operators/_partials/prerequisite_tasks.rst
+
+.. _howto/operator:GoogleDriveToLocalOperator:
+
+GCSToLocalFilesystemOperator
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+:class:`~airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveToLocalOperator` allows you to download
+data from Google Drive to local filesystem.
+
+
+Below is an example of using this operator to download file from Google Drive to Local Filesystem.
+
+.. exampleinclude:: /../../airflow/providers/google/cloud/example_dags/example_gdrive_to_local.py
+    :language: python
+    :dedent: 4
+    :start-after: [START howto_operator_gdrive_download_file_task]
+    :end-before: [END howto_operator_gdrive_download_file_task]

Review comment:
       Yes, I'm already fixed this and previous




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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,95 @@
+# 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 Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):

Review comment:
       Okay. I'll try




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

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



[GitHub] [airflow] Scuall1992 commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: tests/providers/google/cloud/transfers/test_gdrive_to_local.py
##########
@@ -0,0 +1,38 @@
+#
+# 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 unittest import TestCase, mock
+
+from airflow.providers.google.cloud.transfers.gdrive_to_local import GoogleDriveToLocalOperator
+
+TASK_ID = "test-drive-to-local-operator"
+FOLDER_ID = "1234567890qwerty"
+FILE_NAME = "file.pdf"
+OUTPUT_FILE = "out_file.pdf"
+
+
+class TestGoogleDriveToLocalOperator(TestCase):
+    @mock.patch("airflow.providers.google.cloud.transfers.gdrive_to_local.GoogleDriveHook")
+    def test_execute(self, hook_mock):
+        op = GoogleDriveToLocalOperator(
+            task_id=TASK_ID,
+            folder_id=FOLDER_ID,
+            file_name=FILE_NAME,
+            output_file=OUTPUT_FILE,
+        )
+        op.execute(context=None)
+        hook_mock.assert_called_once()

Review comment:
       But how can I mock file_metadata?
   




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

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



[GitHub] [airflow] Scuall1992 commented on pull request #14191: Add new operator gdrive to local

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


   How can I fix this?
   
   Validate providers.yaml files............................................................Failed
   - hook id: provider-yamls
   - exit code: 1
   
   Checking integration duplicates
   Checking completeness of list of {sensors, hooks, operators}
   Checking for duplicates in list of {sensors, hooks, operators}
   Checking completeness of list of transfers
   Checking for duplicates in list of transfers
   Checking connection classes belong to package
   Checking doc files
   Items in the first set but not the second:
   '/docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst'
   
   
   


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata
+
+    def _download_data(self, gdrive_hook: GoogleDriveHook, output_file: str):
+        file_handle = BytesIO()
+        self._set_file_metadata(gdrive_hook=gdrive_hook)
+        file_id = self.file_metadata["id"]
+        request = gdrive_hook.get_media_request(file_id=file_id)
+        gdrive_hook.download_content_from_request(
+            file_handle=file_handle, request=request, chunk_size=104857600
+        )
+
+        with open(output_file, "wb") as f:
+            f.write(file_handle.getbuffer())

Review comment:
       I think it can make sense to move this method to `GoogleDriveHook` so it can be reused in other operators in future. I would suggest the following:
   
   ```py
   def download_file(self, file_id: str, file_handle):
            request = gdrive_hook.get_media_request(file_id=file_id)
            gdrive_hook.download_content_from_request(
                file_handle=file_handle, request=request, chunk_size=104857600
            )
   ```
   
   and then you can simply do in `execute` method:
   ```py
   with open(output_file, "wb" as file:
       gcs_hook.download_file(file_id=self.file_id, file_handle=file)
   ```




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #14191: Add new operator gdrive to local

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/561911205) is cancelling this PR. Building image for the PR has been cancelled


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

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



[GitHub] [airflow] turbaszek commented on pull request #14191: Add new operator gdrive to local

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


   > @turbaszek what the difference with this file https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/transfers/gdrive_to_gcs.py
   > ?
   
   Oh, I missed this one! But I think it would be good to refactor this operator using the new methods to avoid code duplication, WDYT?


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #14191: Add new operator gdrive to local

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



##########
File path: airflow/providers/google/cloud/transfers/gdrive_to_local.py
##########
@@ -0,0 +1,112 @@
+# 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 io import BytesIO
+from typing import Optional, Sequence, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.google.suite.hooks.drive import GoogleDriveHook
+from airflow.utils.decorators import apply_defaults
+
+
+class GoogleDriveToLocalOperator(BaseOperator):
+    """
+    Writes a Google Drive file into local Storage.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:GoogleDriveToGCSOperator`
+
+    :param output_file: Path to downloaded file
+    :type output_file: str
+    :param folder_id: The folder id of the folder in which the Google Drive file resides
+    :type folder_id: str
+    :param filename: The name of the file residing in Google Drive
+    :type filename: str
+    :param drive_id: Optional. The id of the shared Google Drive in which the file resides.
+    :type drive_id: str
+    :param delegate_to: The account to impersonate using domain-wide delegation of authority,
+        if any. For this to work, the service account making the request must have
+        domain-wide delegation enabled.
+    :type delegate_to: str
+    :param impersonation_chain: Optional service account to impersonate using short-term
+        credentials, or chained list of accounts required to get the access_token
+        of the last account in the list, which will be impersonated in the request.
+        If set as a string, the account must grant the originating account
+        the Service Account Token Creator IAM role.
+        If set as a sequence, the identities from the list must grant
+        Service Account Token Creator IAM role to the directly preceding identity, with first
+        account from the list granting this role to the originating account (templated).
+    :type impersonation_chain: Union[str, Sequence[str]]
+    """
+
+    template_fields = [
+        "output_file",
+        "folder_id",
+        "filename",
+        "drive_id",
+        "impersonation_chain",
+    ]
+
+    @apply_defaults
+    def __init__(
+        self,
+        *,
+        output_file: str,
+        filename: str,
+        folder_id: str,
+        drive_id: Optional[str] = None,
+        delegate_to: Optional[str] = None,
+        impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.output_file = output_file
+        self.folder_id = folder_id
+        self.drive_id = drive_id
+        self.filename = filename
+        self.delegate_to = delegate_to
+        self.impersonation_chain = impersonation_chain
+        self.file_metadata = None
+
+    def _set_file_metadata(self, gdrive_hook):
+        if not self.file_metadata:
+            self.file_metadata = gdrive_hook.get_file_id(
+                folder_id=self.folder_id, file_name=self.filename, drive_id=self.drive_id
+            )
+        return self.file_metadata

Review comment:
       @Scuall1992 I think this will be a good approach πŸ‘Œ 




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

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