Source code for airflow.providers.amazon.aws.operators.ecs
## 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__importannotationsimportreimportwarningsfromdatetimeimporttimedeltafromfunctoolsimportcached_propertyfromtypingimportTYPE_CHECKING,Any,Sequencefromairflow.configurationimportconffromairflow.exceptionsimportAirflowException,AirflowProviderDeprecationWarningfromairflow.providers.amazon.aws.exceptionsimportEcsOperatorError,EcsTaskFailToStartfromairflow.providers.amazon.aws.hooks.base_awsimportAwsBaseHookfromairflow.providers.amazon.aws.hooks.ecsimportEcsClusterStates,EcsHook,should_retry_enifromairflow.providers.amazon.aws.hooks.logsimportAwsLogsHookfromairflow.providers.amazon.aws.operators.base_awsimportAwsBaseOperatorfromairflow.providers.amazon.aws.triggers.ecsimport(ClusterActiveTrigger,ClusterInactiveTrigger,TaskDoneTrigger,)fromairflow.providers.amazon.aws.utilsimportvalidate_execute_complete_eventfromairflow.providers.amazon.aws.utils.identifiersimportgenerate_uuidfromairflow.providers.amazon.aws.utils.mixinsimportaws_template_fieldsfromairflow.providers.amazon.aws.utils.task_log_fetcherimportAwsTaskLogFetcherfromairflow.utils.helpersimportprune_dictfromairflow.utils.typesimportNOTSETifTYPE_CHECKING:importboto3fromairflow.modelsimportTaskInstancefromairflow.utils.contextimportContext
[docs]classEcsBaseOperator(AwsBaseOperator[EcsHook]):"""This is the base operator for all Elastic Container Service operators."""
[docs]defclient(self)->boto3.client:"""Create and return the EcsHook's client."""returnself.hook.conn
[docs]defexecute(self,context:Context):"""Must overwrite in child classes."""raiseNotImplementedError("Please implement execute() in subclass")
def_complete_exec_with_cluster_desc(self,context,event=None):"""To be used as trigger callback for operators that return the cluster description."""ifevent["status"]!="success":raiseAirflowException(f"Error while waiting for operation on cluster to complete: {event}")cluster_arn=event.get("arn")# We cannot get the cluster definition from the waiter on success, so we have to query it here.details=self.hook.conn.describe_clusters(clusters=[cluster_arn])["clusters"][0]returndetails
[docs]classEcsCreateClusterOperator(EcsBaseOperator):""" Creates an AWS ECS cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EcsCreateClusterOperator` :param cluster_name: The name of your cluster. If you don't specify a name for your cluster, you create a cluster that's named default. :param create_cluster_kwargs: Extra arguments for Cluster Creation. :param wait_for_completion: If True, waits for creation of the cluster to complete. (default: True) :param waiter_delay: The amount of time in seconds to wait between attempts, if not set then the default waiter value will be used. :param waiter_max_attempts: The maximum number of attempts to be made, if not set then the default waiter value will be used. :param deferrable: If True, the operator will wait asynchronously for the job to complete. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):self.log.info("Creating cluster %r using the following values: %s",self.cluster_name,self.create_cluster_kwargs,)result=self.client.create_cluster(clusterName=self.cluster_name,**self.create_cluster_kwargs)cluster_details=result["cluster"]cluster_state=cluster_details.get("status")ifcluster_state==EcsClusterStates.ACTIVE:# In some circumstances the ECS Cluster is created immediately,# and there is no reason to wait for completion.self.log.info("Cluster %r in state: %r.",self.cluster_name,cluster_state)elifself.deferrable:self.defer(trigger=ClusterActiveTrigger(cluster_arn=cluster_details["clusterArn"],waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region_name=self.region_name,),method_name="_complete_exec_with_cluster_desc",# timeout is set to ensure that if a trigger dies, the timeout does not restart# 60 seconds is added to allow the trigger to exit gracefully (i.e. yield TriggerEvent)timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay+60),)elifself.wait_for_completion:waiter=self.hook.get_waiter("cluster_active")waiter.wait(clusters=[cluster_details["clusterArn"]],WaiterConfig=prune_dict({"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts,}),)returncluster_details
[docs]classEcsDeleteClusterOperator(EcsBaseOperator):""" Deletes an AWS ECS cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EcsDeleteClusterOperator` :param cluster_name: The short name or full Amazon Resource Name (ARN) of the cluster to delete. :param wait_for_completion: If True, waits for creation of the cluster to complete. (default: True) :param waiter_delay: The amount of time in seconds to wait between attempts, if not set then the default waiter value will be used. :param waiter_max_attempts: The maximum number of attempts to be made, if not set then the default waiter value will be used. :param deferrable: If True, the operator will wait asynchronously for the job to complete. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):self.log.info("Deleting cluster %r.",self.cluster_name)result=self.client.delete_cluster(cluster=self.cluster_name)cluster_details=result["cluster"]cluster_state=cluster_details.get("status")ifcluster_state==EcsClusterStates.INACTIVE:# if the cluster doesn't have capacity providers that are associated with it,# the deletion is instantaneous, and we don't need to wait for it.self.log.info("Cluster %r in state: %r.",self.cluster_name,cluster_state)elifself.deferrable:self.defer(trigger=ClusterInactiveTrigger(cluster_arn=cluster_details["clusterArn"],waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region_name=self.region_name,),method_name="_complete_exec_with_cluster_desc",# timeout is set to ensure that if a trigger dies, the timeout does not restart# 60 seconds is added to allow the trigger to exit gracefully (i.e. yield TriggerEvent)timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay+60),)elifself.wait_for_completion:waiter=self.hook.get_waiter("cluster_inactive")waiter.wait(clusters=[cluster_details["clusterArn"]],WaiterConfig=prune_dict({"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts,}),)returncluster_details
[docs]classEcsDeregisterTaskDefinitionOperator(EcsBaseOperator):""" Deregister a task definition on AWS ECS. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EcsDeregisterTaskDefinitionOperator` :param task_definition: The family and revision (family:revision) or full Amazon Resource Name (ARN) of the task definition to deregister. If you use a family name, you must specify a revision. """
def__init__(self,*,task_definition:str,wait_for_completion=NOTSET,waiter_delay=NOTSET,waiter_max_attempts=NOTSET,**kwargs,):ifany(argisnotNOTSETforargin[wait_for_completion,waiter_delay,waiter_max_attempts]):warnings.warn("'wait_for_completion' and waiter related params have no effect and are deprecated, ""please remove them.",AirflowProviderDeprecationWarning,stacklevel=2,)super().__init__(**kwargs)self.task_definition=task_definition
[docs]defexecute(self,context:Context):self.log.info("Deregistering task definition %s.",self.task_definition)result=self.client.deregister_task_definition(taskDefinition=self.task_definition)task_definition_details=result["taskDefinition"]task_definition_arn=task_definition_details["taskDefinitionArn"]self.log.info("Task Definition %r in state: %r.",task_definition_arn,task_definition_details.get("status"))returntask_definition_arn
[docs]classEcsRegisterTaskDefinitionOperator(EcsBaseOperator):""" Register a task definition on AWS ECS. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EcsRegisterTaskDefinitionOperator` :param family: The family name of a task definition to create. :param container_definitions: A list of container definitions in JSON format that describe the different containers that make up your task. :param register_task_kwargs: Extra arguments for Register Task Definition. """
def__init__(self,*,family:str,container_definitions:list[dict],register_task_kwargs:dict|None=None,wait_for_completion=NOTSET,waiter_delay=NOTSET,waiter_max_attempts=NOTSET,**kwargs,):ifany(argisnotNOTSETforargin[wait_for_completion,waiter_delay,waiter_max_attempts]):warnings.warn("'wait_for_completion' and waiter related params have no effect and are deprecated, ""please remove them.",AirflowProviderDeprecationWarning,stacklevel=2,)super().__init__(**kwargs)self.family=familyself.container_definitions=container_definitionsself.register_task_kwargs=register_task_kwargsor{}
[docs]defexecute(self,context:Context):self.log.info("Registering task definition %s using the following values: %s",self.family,self.register_task_kwargs,)self.log.info("Using container definition %s",self.container_definitions)response=self.client.register_task_definition(family=self.family,containerDefinitions=self.container_definitions,**self.register_task_kwargs,)task_definition_details=response["taskDefinition"]task_definition_arn=task_definition_details["taskDefinitionArn"]self.log.info("Task Definition %r in state: %r.",task_definition_arn,task_definition_details.get("status"))context["ti"].xcom_push(key="task_definition_arn",value=task_definition_arn)returntask_definition_arn
[docs]classEcsRunTaskOperator(EcsBaseOperator):""" Execute a task on AWS ECS (Elastic Container Service). .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EcsRunTaskOperator` :param task_definition: the task definition name on Elastic Container Service :param cluster: the cluster name on Elastic Container Service :param overrides: the same parameter that boto3 will receive (templated): https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/ecs.html#ECS.Client.run_task :param aws_conn_id: connection id of AWS credentials / region name. If None, credential boto3 strategy will be used (https://boto3.amazonaws.com/v1/documentation/api/latest/guide/configuration.html). :param region: region name to use in AWS Hook. Override the region in connection (if provided) :param launch_type: the launch type on which to run your task ('EC2', 'EXTERNAL', or 'FARGATE') :param capacity_provider_strategy: the capacity provider strategy to use for the task. When capacity_provider_strategy is specified, the launch_type parameter is omitted. If no capacity_provider_strategy or launch_type is specified, the default capacity provider strategy for the cluster is used. :param group: the name of the task group associated with the task :param placement_constraints: an array of placement constraint objects to use for the task :param placement_strategy: an array of placement strategy objects to use for the task :param platform_version: the platform version on which your task is running :param network_configuration: the network configuration for the task :param tags: a dictionary of tags in the form of {'tagKey': 'tagValue'}. :param awslogs_group: the CloudWatch group where your ECS container logs are stored. Only required if you want logs to be shown in the Airflow UI after your job has finished. :param awslogs_region: the region in which your CloudWatch logs are stored. If None, this is the same as the `region` parameter. If that is also None, this is the default AWS region based on your connection settings. :param awslogs_stream_prefix: the stream prefix that is used for the CloudWatch logs. This is usually based on some custom name combined with the name of the container. Only required if you want logs to be shown in the Airflow UI after your job has finished. :param awslogs_fetch_interval: the interval that the ECS task log fetcher should wait in between each Cloudwatch logs fetches. If deferrable is set to True, that parameter is ignored and waiter_delay is used instead. :param quota_retry: Config if and how to retry the launch of a new ECS task, to handle transient errors. :param reattach: If set to True, will check if the task previously launched by the task_instance is already running. If so, the operator will attach to it instead of starting a new task. This is to avoid relaunching a new task when the connection drops between Airflow and ECS while the task is running (when the Airflow worker is restarted for example). :param number_logs_exception: Number of lines from the last Cloudwatch logs to return in the AirflowException if an ECS task is stopped (to receive Airflow alerts with the logs of what failed in the code running in ECS). :param wait_for_completion: If True, waits for creation of the cluster to complete. (default: True) :param waiter_delay: The amount of time in seconds to wait between attempts, if not set then the default waiter value will be used. :param waiter_max_attempts: The maximum number of attempts to be made, if not set then the default waiter value will be used. :param deferrable: If True, the operator will wait asynchronously for the job to complete. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
def__init__(self,*,task_definition:str,cluster:str,overrides:dict,launch_type:str="EC2",capacity_provider_strategy:list|None=None,group:str|None=None,placement_constraints:list|None=None,placement_strategy:list|None=None,platform_version:str|None=None,network_configuration:dict|None=None,tags:dict|None=None,awslogs_group:str|None=None,awslogs_region:str|None=None,awslogs_stream_prefix:str|None=None,awslogs_fetch_interval:timedelta=timedelta(seconds=30),propagate_tags:str|None=None,quota_retry:dict|None=None,reattach:bool=False,number_logs_exception:int=10,wait_for_completion:bool=True,waiter_delay:int=6,waiter_max_attempts:int=1000000,# Set the default waiter duration to 70 days (attempts*delay)# Airflow execution_timeout handles task timeoutdeferrable:bool=conf.getboolean("operators","default_deferrable",fallback=False),**kwargs,):super().__init__(**kwargs)self.task_definition=task_definitionself.cluster=clusterself.overrides=overridesself.launch_type=launch_typeself.capacity_provider_strategy=capacity_provider_strategyself.group=groupself.placement_constraints=placement_constraintsself.placement_strategy=placement_strategyself.platform_version=platform_versionself.network_configuration=network_configurationself.tags=tagsself.awslogs_group=awslogs_groupself.awslogs_stream_prefix=awslogs_stream_prefixself.awslogs_region=awslogs_regionself.awslogs_fetch_interval=awslogs_fetch_intervalself.propagate_tags=propagate_tagsself.reattach=reattachself.number_logs_exception=number_logs_exceptionifself.awslogs_regionisNone:self.awslogs_region=self.region_nameself.arn:str|None=Noneself._started_by:str|None=Noneself.retry_args=quota_retryself.task_log_fetcher:AwsTaskLogFetcher|None=Noneself.wait_for_completion=wait_for_completionself.waiter_delay=waiter_delayself.waiter_max_attempts=waiter_max_attemptsself.deferrable=deferrableifself._aws_logs_enabled()andnotself.wait_for_completion:self.log.warning("Trying to get logs without waiting for the task to complete is undefined behavior.")@staticmethoddef_get_ecs_task_id(task_arn:str|None)->str|None:iftask_arnisNone:returnNonereturntask_arn.split("/")[-1]
[docs]defexecute(self,context):self.log.info("Running ECS Task - Task definition: %s - on cluster %s",self.task_definition,self.cluster)self.log.info("EcsOperator overrides: %s",self.overrides)ifself.reattach:# Generate deterministic UUID which refers to unique TaskInstanceKeyti:TaskInstance=context["ti"]self._started_by=generate_uuid(*map(str,ti.key.primary))self.log.info("Try to find run with startedBy=%r",self._started_by)self._try_reattach_task(started_by=self._started_by)ifnotself.arn:# start the task except if we reattached to an existing one just before.self._start_task()ifself.do_xcom_push:self.xcom_push(context,key="ecs_task_arn",value=self.arn)ifself.deferrable:self.defer(trigger=TaskDoneTrigger(cluster=self.cluster,task_arn=self.arn,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region=self.region_name,log_group=self.awslogs_group,log_stream=self._get_logs_stream_name(),),method_name="execute_complete",# timeout is set to ensure that if a trigger dies, the timeout does not restart# 60 seconds is added to allow the trigger to exit gracefully (i.e. yield TriggerEvent)timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay+60),)# self.defer raises a special exception, so execution stops here in this case.ifnotself.wait_for_completion:returnifself._aws_logs_enabled():self.log.info("Starting ECS Task Log Fetcher")self.task_log_fetcher=self._get_task_log_fetcher()self.task_log_fetcher.start()try:self._wait_for_task_ended()finally:self.task_log_fetcher.stop()self.task_log_fetcher.join()else:self._wait_for_task_ended()self._after_execution()ifself.do_xcom_pushandself.task_log_fetcher:returnself.task_log_fetcher.get_last_log_message()else:returnNone
[docs]defexecute_complete(self,context:Context,event:dict[str,Any]|None=None)->str|None:event=validate_execute_complete_event(event)ifevent["status"]!="success":raiseAirflowException(f"Error in task execution: {event}")self.arn=event["task_arn"]# restore arn to its updated value, needed for next stepsself._after_execution()ifself._aws_logs_enabled():# same behavior as non-deferrable mode, return last line of logs of the task.logs_client=AwsLogsHook(aws_conn_id=self.aws_conn_id,region_name=self.region_name).connone_log=logs_client.get_log_events(logGroupName=self.awslogs_group,logStreamName=self._get_logs_stream_name(),startFromHead=False,limit=1,)iflen(one_log["events"])>0:returnone_log["events"][0]["message"]returnNone
def_after_execution(self):self._check_success_task()def_start_task(self):run_opts={"cluster":self.cluster,"taskDefinition":self.task_definition,"overrides":self.overrides,"startedBy":self._started_byorself.owner,}ifself.capacity_provider_strategy:run_opts["capacityProviderStrategy"]=self.capacity_provider_strategyelifself.launch_type:run_opts["launchType"]=self.launch_typeifself.platform_versionisnotNone:run_opts["platformVersion"]=self.platform_versionifself.groupisnotNone:run_opts["group"]=self.groupifself.placement_constraintsisnotNone:run_opts["placementConstraints"]=self.placement_constraintsifself.placement_strategyisnotNone:run_opts["placementStrategy"]=self.placement_strategyifself.network_configurationisnotNone:run_opts["networkConfiguration"]=self.network_configurationifself.tagsisnotNone:run_opts["tags"]=[{"key":k,"value":v}for(k,v)inself.tags.items()]ifself.propagate_tagsisnotNone:run_opts["propagateTags"]=self.propagate_tagsresponse=self.client.run_task(**run_opts)failures=response["failures"]iflen(failures)>0:raiseEcsOperatorError(failures,response)self.log.info("ECS Task started: %s",response)self.arn=response["tasks"][0]["taskArn"]self.log.info("ECS task ID is: %s",self._get_ecs_task_id(self.arn))def_try_reattach_task(self,started_by:str):ifnotstarted_by:raiseAirflowException("`started_by` should not be empty or None")list_tasks_resp=self.client.list_tasks(cluster=self.cluster,desiredStatus="RUNNING",startedBy=started_by)running_tasks=list_tasks_resp["taskArns"]ifrunning_tasks:iflen(running_tasks)>1:self.log.warning("Found more then one previously launched tasks: %s",running_tasks)self.arn=running_tasks[0]self.log.info("Reattaching previously launched task: %s",self.arn)else:self.log.info("No active previously launched task found to reattach")def_wait_for_task_ended(self)->None:ifnotself.clientornotself.arn:returnwaiter=self.client.get_waiter("tasks_stopped")waiter.wait(cluster=self.cluster,tasks=[self.arn],WaiterConfig={"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts,},)def_aws_logs_enabled(self):returnself.awslogs_groupandself.awslogs_stream_prefixdef_get_logs_stream_name(self)->str:returnf"{self.awslogs_stream_prefix}/{self._get_ecs_task_id(self.arn)}"def_get_task_log_fetcher(self)->AwsTaskLogFetcher:ifnotself.awslogs_group:raiseValueError("must specify awslogs_group to fetch task logs")returnAwsTaskLogFetcher(aws_conn_id=self.aws_conn_id,region_name=self.awslogs_region,log_group=self.awslogs_group,log_stream_name=self._get_logs_stream_name(),fetch_interval=self.awslogs_fetch_interval,logger=self.log,)@AwsBaseHook.retry(should_retry_eni)def_check_success_task(self)->None:ifnotself.clientornotself.arn:returnresponse=self.client.describe_tasks(cluster=self.cluster,tasks=[self.arn])self.log.info("ECS Task stopped, check status: %s",response)iflen(response.get("failures",[]))>0:raiseAirflowException(response)fortaskinresponse["tasks"]:iftask.get("stopCode","")=="TaskFailedToStart":# Reset task arn here otherwise the retry run will not start# a new task but keep polling the old dead one# I'm not resetting it for other exceptions here because# EcsTaskFailToStart is the only exception that's being retried at the momentself.arn=NoneraiseEcsTaskFailToStart(f"The task failed to start due to: {task.get('stoppedReason','')}")# This is a `stoppedReason` that indicates a task has not# successfully finished, but there is no other indication of failure# in the response.# https://docs.aws.amazon.com/AmazonECS/latest/developerguide/stopped-task-errors.htmlifre.match(r"Host EC2 \(instance .+?\) (stopped|terminated)\.",task.get("stoppedReason","")):raiseAirflowException(f"The task was stopped because the host instance terminated:"f" {task.get('stoppedReason','')}")containers=task["containers"]forcontainerincontainers:ifcontainer.get("lastStatus")=="STOPPED"andcontainer.get("exitCode",1)!=0:ifself.task_log_fetcher:last_logs="\n".join(self.task_log_fetcher.get_last_log_messages(self.number_logs_exception))raiseAirflowException(f"This task is not in success state - last {self.number_logs_exception} "f"logs from Cloudwatch:\n{last_logs}")else:raiseAirflowException(f"This task is not in success state {task}")elifcontainer.get("lastStatus")=="PENDING":raiseAirflowException(f"This task is still pending {task}")elif"error"incontainer.get("reason","").lower():raiseAirflowException(f"This containers encounter an error during launching: "f"{container.get('reason','').lower()}")
[docs]defon_kill(self)->None:ifnotself.clientornotself.arn:returnifself.task_log_fetcher:self.task_log_fetcher.stop()response=self.client.stop_task(cluster=self.cluster,task=self.arn,reason="Task killed by the user")self.log.info(response)