Skip to content

Commit

Permalink
Add LocalFilesystemToGoogleDriveOperator (#22219)
Browse files Browse the repository at this point in the history
*  Add `LocalFilesystemToGoogleDriveOperator`

Co-authored-by: Jarek Potiuk <[email protected]>
Co-authored-by: Tomek Urbaszek <[email protected]>
Co-authored-by: Josh Fell <[email protected]>
  • Loading branch information
4 people committed Mar 21, 2022
1 parent 14e6b65 commit 7b851ed
Show file tree
Hide file tree
Showing 5 changed files with 280 additions and 0 deletions.
4 changes: 4 additions & 0 deletions airflow/providers/google/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -824,6 +824,10 @@ transfers:
target-integration-name: Local
how-to-guide: /docs/apache-airflow-providers-google/operators/transfer/gdrive_to_local.rst
python-module: airflow.providers.google.cloud.transfers.gdrive_to_local
- source-integration-name: Local
target-integration-name: Google Drive
how-to-guide: /docs/apache-airflow-providers-google/operators/transfer/local_to_drive.rst
python-module: airflow.providers.google.suite.transfers.local_to_drive
- source-integration-name: Salesforce
target-integration-name: Google Cloud Storage (GCS)
how-to-guide: /docs/apache-airflow-providers-google/operators/transfer/salesforce_to_gcs.rst
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
#
# 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.
"""
Example DAG using LocalFilesystemToGoogleDriveOperator.
"""

from datetime import datetime
from pathlib import Path

from airflow import models
from airflow.providers.google.suite.transfers.local_to_drive import LocalFilesystemToGoogleDriveOperator

SINGLE_FILE_LOCAL_PATHS = [Path("test1")]
MULTIPLE_FILES_LOCAL_PATHS = [Path("test1"), Path("test2")]
DRIVE_FOLDER = Path("test-folder")

with models.DAG(
"example_local_to_drive",
schedule_interval='@once', # Override to match your needs
start_date=datetime(2021, 1, 1),
catchup=False,
tags=["example"],
) as dag:
# [START howto_operator_local_to_drive_upload_single_file]
upload_single_file = LocalFilesystemToGoogleDriveOperator(
task_id="upload_single_file",
local_paths=SINGLE_FILE_LOCAL_PATHS,
drive_folder=DRIVE_FOLDER,
)
# [END howto_operator_local_to_drive_upload_single_file]

# [START howto_operator_local_to_drive_upload_multiple_files]
upload_multiple_files = LocalFilesystemToGoogleDriveOperator(
task_id="upload_multiple_files",
local_paths=MULTIPLE_FILES_LOCAL_PATHS,
drive_folder=DRIVE_FOLDER,
ignore_if_missing=True,
)
# [END howto_operator_local_to_drive_upload_multiple_files]

upload_single_file >> upload_multiple_files
130 changes: 130 additions & 0 deletions airflow/providers/google/suite/transfers/local_to_drive.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
# 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.
"""This file contains Google Drive operators"""

import os
from pathlib import Path
from typing import TYPE_CHECKING, List, Optional, Sequence, Union

from airflow.exceptions import AirflowFailException
from airflow.models import BaseOperator
from airflow.providers.google.suite.hooks.drive import GoogleDriveHook

if TYPE_CHECKING:
from airflow.utils.context import Context


class LocalFilesystemToGoogleDriveOperator(BaseOperator):
"""
Upload a list of files to a Google Drive folder.
This operator uploads a list of local files to a Google Drive folder.
The local files can be deleted after upload (optional)
.. seealso::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:LocalFilesystemToGoogleDriveOperator`
:param local_paths: Python list of local file paths
:param drive_folder: path of the Drive folder
:param gcp_conn_id: Airflow Connection ID for GCP
:param delete: should the local files be deleted after upload?
:param ignore_if_missing: if True, then don't fail even if all files
can't be uploaded.
:param chunk_size: File will be uploaded in chunks of this many bytes. Only
used if resumable=True. Pass in a value of -1 if the file is to be
uploaded as a single chunk. Note that Google App Engine has a 5MB limit
on request size, so you should never set your chunk size larger than 5MB,
or to -1.
:param resumable: True if this is a resumable upload. False means upload
in a single request.
: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.
: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
:return: Remote file ids after upload
:rtype: Sequence[str]
"""

template_fields = (
'local_paths',
'drive_folder',
)

def __init__(
self,
local_paths: Union[Sequence[Path], Sequence[str]],
drive_folder: Union[Path, str],
gcp_conn_id: str = "google_cloud_default",
delete: bool = False,
ignore_if_missing: bool = False,
chunk_size: int = 100 * 1024 * 1024,
resumable: bool = False,
delegate_to: Optional[str] = None,
impersonation_chain: Optional[Union[str, Sequence[str]]] = None,
**kwargs,
) -> None:
super().__init__(**kwargs)
self.local_paths = local_paths
self.drive_folder = drive_folder
self.gcp_conn_id = gcp_conn_id
self.delete = delete
self.ignore_if_missing = ignore_if_missing
self.chunk_size = chunk_size
self.resumable = resumable
self.delegate_to = delegate_to
self.impersonation_chain = impersonation_chain

def execute(self, context: "Context") -> List[str]:
hook = GoogleDriveHook(
gcp_conn_id=self.gcp_conn_id,
delegate_to=self.delegate_to,
impersonation_chain=self.impersonation_chain,
)

remote_file_ids = []

for local_path in self.local_paths:
self.log.info("Uploading file to Google Drive: %s", local_path)

try:
remote_file_id = hook.upload_file(
local_location=str(local_path),
remote_location=str(Path(self.drive_folder) / Path(local_path).name),
chunk_size=self.chunk_size,
resumable=self.resumable,
)

remote_file_ids.append(remote_file_id)

if self.delete:
os.remove(local_path)
self.log.info("Deleted local file: %s", local_path)
except FileNotFoundError:
self.log.warning("File can't be found: %s", local_path)
except OSError:
self.log.warning("An OSError occurred for file: %s", local_path)

if not self.ignore_if_missing and len(remote_file_ids) < len(self.local_paths):
raise AirflowFailException("Some files couldn't be uploaded")
return remote_file_ids
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
.. 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.
Upload data from Local Filesystem to Google Drive
=================================================
The `Google Drive <https://drive.google.com/>`__ is used to store large data from various applications.
This page shows how to upload data from local filesystem to Google Drive.

Prerequisite Tasks
^^^^^^^^^^^^^^^^^^

.. include::/operators/_partials/prerequisite_tasks.rst
.. _howto/operator:LocalFilesystemToGoogleDriveOperator:

LocalFilesystemToGoogleDriveOperator
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~

:class:`~airflow.providers.google.suite.transfers.local_to_drive.LocalFilesystemToGoogleDriveOperator` allows you to upload
data from local filesystem to GoogleDrive.

When you use this operator, you can upload a list of files.
53 changes: 53 additions & 0 deletions tests/providers/google/suite/transfers/test_local_to_drive.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
# 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 pathlib import Path
from unittest import mock

from airflow.providers.google.suite.transfers.local_to_drive import LocalFilesystemToGoogleDriveOperator

GCP_CONN_ID = "test"
DRIVE_FOLDER = Path("test_folder")
LOCAL_PATHS = [Path("test1"), Path("test2")]
REMOTE_FILE_IDS = ["rtest1", "rtest2"]


class TestLocalFilesystemToGoogleDriveOperator:
@mock.patch("airflow.providers.google.suite.transfers.local_to_drive.GoogleDriveHook")
def test_execute(self, mock_hook):
context = {}
mock_hook.return_value.upload_file.return_value = REMOTE_FILE_IDS
op = LocalFilesystemToGoogleDriveOperator(
task_id="test_task", local_paths=LOCAL_PATHS, drive_folder=DRIVE_FOLDER, gcp_conn_id=GCP_CONN_ID
)
op.execute(context)

calls = [
mock.call(
local_location="test1",
remote_location="test_folder/test1",
chunk_size=100 * 1024 * 1024,
resumable=False,
),
mock.call(
local_location="test2",
remote_location="test_folder/test2",
chunk_size=100 * 1024 * 1024,
resumable=False,
),
]
mock_hook.return_value.upload_file.assert_has_calls(calls)

0 comments on commit 7b851ed

Please sign in to comment.