Source code for airflow.providers.amazon.aws.transfers.s3_to_dynamodb
## 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__importannotationsfromtypingimportTYPE_CHECKING,Any,Literal,Sequence,TypedDictfrombotocore.exceptionsimportClientError,WaiterErrorfromairflow.exceptionsimportAirflowExceptionfromairflow.modelsimportBaseOperatorfromairflow.providers.amazon.aws.hooks.dynamodbimportDynamoDBHookifTYPE_CHECKING:fromairflow.utils.contextimportContext
[docs]classS3ToDynamoDBOperator(BaseOperator):""" Load Data from S3 into a DynamoDB. Data stored in S3 can be uploaded to a new or existing DynamoDB. Supported file formats CSV, DynamoDB JSON and Amazon ION. :param s3_bucket: The S3 bucket that is imported :param s3_key: Key prefix that imports single or multiple objects from S3 :param dynamodb_table_name: Name of the table that shall be created :param dynamodb_key_schema: Primary key and sort key. Each element represents one primary key attribute. AttributeName is the name of the attribute. KeyType is the role for the attribute. Valid values HASH or RANGE :param dynamodb_attributes: Name of the attributes of a table. AttributeName is the name for the attribute AttributeType is the data type for the attribute. Valid values for AttributeType are S - attribute is of type String N - attribute is of type Number B - attribute is of type Binary :param dynamodb_tmp_table_prefix: Prefix for the temporary DynamoDB table :param delete_on_error: If set, the new DynamoDB table will be deleted in case of import errors :param use_existing_table: Whether to import to an existing non new DynamoDB table. If set to true data is loaded first into a temporary DynamoDB table (using the AWS ImportTable Service), then retrieved as chunks into memory and loaded into the target table. If set to false, a new DynamoDB table will be created and S3 data is bulk loaded by the AWS ImportTable Service. :param input_format: The format for the imported data. Valid values for InputFormat are CSV, DYNAMODB_JSON or ION :param billing_mode: Billing mode for the table. Valid values are PROVISIONED or PAY_PER_REQUEST :param on_demand_throughput: Extra options for maximum number of read and write units :param import_table_kwargs: Any additional optional import table parameters to pass, such as ClientToken, InputCompressionType, or InputFormatOptions. See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/dynamodb/client/import_table.html :param import_table_creation_kwargs: Any additional optional import table creation parameters to pass, such as ProvisionedThroughput, SSESpecification, or GlobalSecondaryIndexes. See: https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/dynamodb/client/import_table.html :param wait_for_completion: Whether to wait for cluster to stop :param check_interval: Time in seconds to wait between status checks :param max_attempts: Maximum number of attempts to check for job completion :param aws_conn_id: The reference to the AWS connection details """
def_load_into_new_table(self,table_name:str,delete_on_error:bool)->str:""" Import S3 key or keys into a new DynamoDB table. :param table_name: Name of the table that shall be created :param delete_on_error: If set, the new DynamoDB table will be deleted in case of import errors :return: The Amazon resource number (ARN) """dynamodb_hook=DynamoDBHook(aws_conn_id=self.aws_conn_id)client=dynamodb_hook.clientimport_table_config=self.import_table_kwargsor{}import_table_creation_config=self.import_table_creation_kwargsor{}try:response=client.import_table(S3BucketSource={"S3Bucket":self.s3_bucket,"S3KeyPrefix":self.s3_key,},InputFormat=self.input_format,TableCreationParameters={"TableName":table_name,"AttributeDefinitions":self.dynamodb_attributes,"KeySchema":self.dynamodb_key_schema,"BillingMode":self.billing_mode,**import_table_creation_config,},**import_table_config,)exceptClientErrorase:self.log.error("Error: failed to load from S3 into DynamoDB table. Error: %s",str(e))raiseAirflowException(f"S3 load into DynamoDB table failed with error: {e}")ifresponse["ImportTableDescription"]["ImportStatus"]=="FAILED":raiseAirflowException("S3 into Dynamodb job creation failed. Code: "f"{response['ImportTableDescription']['FailureCode']}. "f"Failure: {response['ImportTableDescription']['FailureMessage']}")ifself.wait_for_completion:self.log.info("Waiting for S3 into Dynamodb job to complete")waiter=dynamodb_hook.get_waiter("import_table")try:waiter.wait(ImportArn=response["ImportTableDescription"]["ImportArn"],WaiterConfig={"Delay":self.check_interval,"MaxAttempts":self.max_attempts},)exceptWaiterError:status,error_code,error_msg=dynamodb_hook.get_import_status(response["ImportTableDescription"]["ImportArn"])ifdelete_on_error:client.delete_table(TableName=table_name)raiseAirflowException(f"S3 import into Dynamodb job failed: Status: {status}. Error: {error_code}. Error message: {error_msg}")returnresponse["ImportTableDescription"]["ImportArn"]def_load_into_existing_table(self)->str:""" Import S3 key or keys in an existing DynamoDB table. :return:The Amazon resource number (ARN) """ifnotself.wait_for_completion:raiseValueError("wait_for_completion must be set to True when loading into an existing table")table_keys=[key["AttributeName"]forkeyinself.dynamodb_key_schema]dynamodb_hook=DynamoDBHook(aws_conn_id=self.aws_conn_id,table_name=self.dynamodb_table_name,table_keys=table_keys)client=dynamodb_hook.clientself.log.info("Loading from S3 into a tmp DynamoDB table %s",self.tmp_table_name)self._load_into_new_table(table_name=self.tmp_table_name,delete_on_error=self.delete_on_error)total_items=0try:paginator=client.get_paginator("scan")paginate=paginator.paginate(TableName=self.tmp_table_name,Select="ALL_ATTRIBUTES",ReturnConsumedCapacity="NONE",ConsistentRead=True,)self.log.info("Loading data from %s to %s DynamoDB table",self.tmp_table_name,self.dynamodb_table_name)forpageinpaginate:total_items+=page.get("Count",0)dynamodb_hook.write_batch_data(items=page["Items"])self.log.info("Number of items loaded: %s",total_items)finally:self.log.info("Delete tmp DynamoDB table %s",self.tmp_table_name)client.delete_table(TableName=self.tmp_table_name)returndynamodb_hook.get_conn().Table(self.dynamodb_table_name).table_arn
[docs]defexecute(self,context:Context)->str:""" Execute S3 to DynamoDB Job from Airflow. :param context: The current context of the task instance :return: The Amazon resource number (ARN) """ifself.use_existing_table:self.log.info("Loading from S3 into new DynamoDB table %s",self.dynamodb_table_name)returnself._load_into_existing_table()self.log.info("Loading from S3 into existing DynamoDB table %s",self.dynamodb_table_name)returnself._load_into_new_table(table_name=self.dynamodb_table_name,delete_on_error=self.delete_on_error)