Source code for airflow.providers.google.cloud.transfers.s3_to_gcs
## 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__future__importannotationsfromdatetimeimportdatetimefromtempfileimportNamedTemporaryFilefromtypingimportTYPE_CHECKING,Any,Sequencefromairflow.configurationimportconffromairflow.exceptionsimportAirflowExceptionfromairflow.providers.amazon.aws.hooks.s3importS3Hookfromairflow.providers.google.cloud.hooks.cloud_storage_transfer_serviceimport(ACCESS_KEY_ID,AWS_ACCESS_KEY,AWS_S3_DATA_SOURCE,AWS_SECRET_ACCESS_KEY,BUCKET_NAME,GCS_DATA_SINK,INCLUDE_PREFIXES,OBJECT_CONDITIONS,OVERWRITE_OBJECTS_ALREADY_EXISTING_IN_SINK,PATH,PROJECT_ID,SCHEDULE,SCHEDULE_END_DATE,SCHEDULE_START_DATE,STATUS,TRANSFER_OPTIONS,TRANSFER_SPEC,CloudDataTransferServiceHook,GcpTransferJobsStatus,)fromairflow.providers.google.cloud.hooks.gcsimportGCSHook,_parse_gcs_url,gcs_object_is_directoryfromairflow.providers.google.cloud.triggers.cloud_storage_transfer_serviceimport(CloudStorageTransferServiceCreateJobsTrigger,)try:fromairflow.providers.amazon.aws.operators.s3importS3ListOperatorexceptImportError:fromairflow.providers.amazon.aws.operators.s3_listimportS3ListOperator# type: ignore[no-redef]ifTYPE_CHECKING:fromairflow.utils.contextimportContext
[docs]classS3ToGCSOperator(S3ListOperator):""" Synchronizes an S3 key, possibly a prefix, with a Google Cloud Storage destination path. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:S3ToGCSOperator` :param bucket: The S3 bucket where to find the objects. (templated) :param prefix: Prefix string which filters objects whose name begin with such prefix. (templated) :param apply_gcs_prefix: (Optional) Whether to replace source objects' path by given GCS destination path. If apply_gcs_prefix is False (default), then objects from S3 will be copied to GCS bucket into a given GSC path and the source path will be place inside. For example, <s3_bucket><s3_prefix><content> => <gcs_prefix><s3_prefix><content> If apply_gcs_prefix is True, then objects from S3 will be copied to GCS bucket into a given GCS path and the source path will be omitted. For example: <s3_bucket><s3_prefix><content> => <gcs_prefix><content> :param delimiter: the delimiter marks key hierarchy. (templated) :param aws_conn_id: The source S3 connection :param verify: Whether or not to verify SSL certificates for S3 connection. By default SSL certificates are verified. You can provide the following values: - ``False``: do not validate SSL certificates. SSL will still be used (unless use_ssl is False), but SSL certificates will not be verified. - ``path/to/cert/bundle.pem``: A filename of the CA cert bundle to uses. You can specify this argument if you want to use a different CA cert bundle than the one used by botocore. :param gcp_conn_id: (Optional) The connection ID used to connect to Google Cloud. :param dest_gcs: The destination Google Cloud Storage bucket and prefix where you want to store the files. (templated) :param replace: Whether you want to replace existing destination files or not. :param gzip: Option to compress file for upload. Parameter ignored in deferrable mode. :param google_impersonation_chain: Optional Google 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). :param deferrable: Run operator in the deferrable mode :param poll_interval: time in seconds between polling for job completion. The value is considered only when running in deferrable mode. Must be greater than 0. **Example**: .. code-block:: python s3_to_gcs_op = S3ToGCSOperator( task_id="s3_to_gcs_example", bucket="my-s3-bucket", prefix="data/customers-201804", gcp_conn_id="google_cloud_default", dest_gcs="gs://my.gcs.bucket/some/customers/", replace=False, gzip=True, dag=my_dag, ) Note that ``bucket``, ``prefix``, ``delimiter`` and ``dest_gcs`` are templated, so you can use variables in them if you wish. """
def__init__(self,*,bucket,prefix="",apply_gcs_prefix=False,delimiter="",aws_conn_id="aws_default",verify=None,gcp_conn_id="google_cloud_default",dest_gcs=None,replace=False,gzip=False,google_impersonation_chain:str|Sequence[str]|None=None,deferrable=conf.getboolean("operators","default_deferrable",fallback=False),poll_interval:int=10,**kwargs,):super().__init__(bucket=bucket,prefix=prefix,delimiter=delimiter,aws_conn_id=aws_conn_id,**kwargs)self.apply_gcs_prefix=apply_gcs_prefixself.gcp_conn_id=gcp_conn_idself.dest_gcs=dest_gcsself.replace=replaceself.verify=verifyself.gzip=gzipself.google_impersonation_chain=google_impersonation_chainself.deferrable=deferrableifpoll_interval<=0:raiseValueError("Invalid value for poll_interval. Expected value greater than 0")self.poll_interval=poll_intervaldef_check_inputs(self)->None:ifself.dest_gcsandnotgcs_object_is_directory(self.dest_gcs):self.log.info("Destination Google Cloud Storage path is not a valid "'"directory", define a path that ends with a slash "/" or '"leave it empty for the root of the bucket.")raiseAirflowException('The destination Google Cloud Storage path must end with a slash "/" or be empty.')
[docs]defexecute(self,context:Context):self._check_inputs()# use the super method to list all the files in an S3 bucket/keys3_objects=super().execute(context)gcs_hook=GCSHook(gcp_conn_id=self.gcp_conn_id,impersonation_chain=self.google_impersonation_chain,)ifnotself.replace:s3_objects=self.exclude_existing_objects(s3_objects=s3_objects,gcs_hook=gcs_hook)s3_hook=S3Hook(aws_conn_id=self.aws_conn_id,verify=self.verify)ifnots3_objects:self.log.info("In sync, no files needed to be uploaded to Google Cloud Storage")elifself.deferrable:self.transfer_files_async(s3_objects,gcs_hook,s3_hook)else:self.transfer_files(s3_objects,gcs_hook,s3_hook)returns3_objects
[docs]defexclude_existing_objects(self,s3_objects:list[str],gcs_hook:GCSHook)->list[str]:"""Excludes from the list objects that already exist in GCS bucket."""bucket_name,object_prefix=_parse_gcs_url(self.dest_gcs)existing_gcs_objects=set(gcs_hook.list(bucket_name,prefix=object_prefix))s3_paths=set(self.gcs_to_s3_object(gcs_object=gcs_object)forgcs_objectinexisting_gcs_objects)s3_objects_reduced=list(set(s3_objects)-s3_paths)ifs3_objects_reduced:self.log.info("%s files are going to be synced: %s.",len(s3_objects_reduced),s3_objects_reduced)else:self.log.info("There are no new files to sync. Have a nice day!")returns3_objects_reduced
[docs]defs3_to_gcs_object(self,s3_object:str)->str:""" Transforms S3 path to GCS path according to the operator's logic. If apply_gcs_prefix == True then <s3_prefix><content> => <gcs_prefix><content> If apply_gcs_prefix == False then <s3_prefix><content> => <gcs_prefix><s3_prefix><content> """gcs_bucket,gcs_prefix=_parse_gcs_url(self.dest_gcs)ifself.apply_gcs_prefix:gcs_object=s3_object.replace(self.prefix,gcs_prefix,1)returngcs_objectreturngcs_prefix+s3_object
[docs]defgcs_to_s3_object(self,gcs_object:str)->str:""" Transforms GCS path to S3 path according to the operator's logic. If apply_gcs_prefix == True then <gcs_prefix><content> => <s3_prefix><content> If apply_gcs_prefix == False then <gcs_prefix><s3_prefix><content> => <s3_prefix><content> """gcs_bucket,gcs_prefix=_parse_gcs_url(self.dest_gcs)s3_object=gcs_object.replace(gcs_prefix,"",1)ifself.apply_gcs_prefix:returnself.prefix+s3_objectreturns3_object
[docs]deftransfer_files(self,s3_objects:list[str],gcs_hook:GCSHook,s3_hook:S3Hook)->None:ifs3_objects:dest_gcs_bucket,dest_gcs_object_prefix=_parse_gcs_url(self.dest_gcs)forobjins3_objects:# GCS hook builds its own in-memory file, so we have to create# and pass the pathfile_object=s3_hook.get_key(obj,self.bucket)withNamedTemporaryFile(mode="wb",delete=True)asfile:file_object.download_fileobj(file)file.flush()gcs_file=self.s3_to_gcs_object(s3_object=obj)gcs_hook.upload(dest_gcs_bucket,gcs_file,file.name,gzip=self.gzip)self.log.info("All done, uploaded %d files to Google Cloud Storage",len(s3_objects))
[docs]deftransfer_files_async(self,files:list[str],gcs_hook:GCSHook,s3_hook:S3Hook)->None:"""Submits Google Cloud Storage Transfer Service job to copy files from AWS S3 to GCS."""ifnotlen(files):raiseValueError("List of transferring files cannot be empty")job_names=self.submit_transfer_jobs(files=files,gcs_hook=gcs_hook,s3_hook=s3_hook)self.defer(trigger=CloudStorageTransferServiceCreateJobsTrigger(project_id=gcs_hook.project_id,job_names=job_names,poll_interval=self.poll_interval,),method_name="execute_complete",)
[docs]defsubmit_transfer_jobs(self,files:list[str],gcs_hook:GCSHook,s3_hook:S3Hook)->list[str]:now=datetime.utcnow()one_time_schedule={"day":now.day,"month":now.month,"year":now.year}gcs_bucket,gcs_prefix=_parse_gcs_url(self.dest_gcs)config=s3_hook.conn_configbody:dict[str,Any]={PROJECT_ID:gcs_hook.project_id,STATUS:GcpTransferJobsStatus.ENABLED,SCHEDULE:{SCHEDULE_START_DATE:one_time_schedule,SCHEDULE_END_DATE:one_time_schedule,},TRANSFER_SPEC:{AWS_S3_DATA_SOURCE:{BUCKET_NAME:self.bucket,AWS_ACCESS_KEY:{ACCESS_KEY_ID:config.aws_access_key_id,AWS_SECRET_ACCESS_KEY:config.aws_secret_access_key,},},OBJECT_CONDITIONS:{INCLUDE_PREFIXES:[],},GCS_DATA_SINK:{BUCKET_NAME:gcs_bucket,PATH:gcs_prefix},TRANSFER_OPTIONS:{OVERWRITE_OBJECTS_ALREADY_EXISTING_IN_SINK:self.replace,},},}# max size of the field 'transfer_job.transfer_spec.object_conditions.include_prefixes' is 1000,# that's why we submit multiple jobs transferring 1000 files each.# See documentation below# https://cloud.google.com/storage-transfer/docs/reference/rest/v1/TransferSpec#ObjectConditionschunk_size=self.transfer_job_max_files_numberjob_names=[]transfer_hook=self.get_transfer_hook()foriinrange(0,len(files),chunk_size):files_chunk=files[i:i+chunk_size]body[TRANSFER_SPEC][OBJECT_CONDITIONS][INCLUDE_PREFIXES]=files_chunkjob=transfer_hook.create_transfer_job(body=body)s="s"iflen(files_chunk)>1else""self.log.info(f"Submitted job {job['name']} to transfer {len(files_chunk)} file{s}")job_names.append(job["name"])iflen(files)>chunk_size:js="s"iflen(job_names)>1else""fs="s"iflen(files)>1else""self.log.info(f"Overall submitted {len(job_names)} job{js} to transfer {len(files)} file{fs}")returnjob_names
[docs]defexecute_complete(self,context:Context,event:dict[str,Any])->None:""" Callback for when the trigger fires - returns immediately. Relies on trigger to throw an exception, otherwise it assumes execution was successful. """ifevent["status"]=="error":raiseAirflowException(event["message"])self.log.info("%s completed with response %s ",self.task_id,event["message"])