Source code for airflow.providers.amazon.aws.operators.eks
# 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 module contains Amazon EKS operators."""from__future__importannotationsimportloggingimportwarningsfromastimportliteral_evalfromdatetimeimporttimedeltafromfunctoolsimportcached_propertyfromtypingimportTYPE_CHECKING,Any,List,Sequence,castfrombotocore.exceptionsimportClientError,WaiterErrorfromdeprecatedimportdeprecatedfromairflow.configurationimportconffromairflow.exceptionsimportAirflowException,AirflowProviderDeprecationWarningfromairflow.modelsimportBaseOperatorfromairflow.providers.amazon.aws.hooks.eksimportEksHookfromairflow.providers.amazon.aws.triggers.eksimport(EksCreateClusterTrigger,EksCreateFargateProfileTrigger,EksCreateNodegroupTrigger,EksDeleteClusterTrigger,EksDeleteFargateProfileTrigger,EksDeleteNodegroupTrigger,)fromairflow.providers.amazon.aws.utilsimportvalidate_execute_complete_eventfromairflow.providers.amazon.aws.utils.waiter_with_loggingimportwaitfromairflow.providers.cncf.kubernetes.utils.pod_managerimportOnFinishActiontry:fromairflow.providers.cncf.kubernetes.operators.podimportKubernetesPodOperatorexceptImportError:# preserve backward compatibility for older versions of cncf.kubernetes providerfromairflow.providers.cncf.kubernetes.operators.kubernetes_podimportKubernetesPodOperatorifTYPE_CHECKING:fromairflow.utils.contextimportContext
def_create_compute(compute:str|None,cluster_name:str,aws_conn_id:str|None,region:str|None,waiter_delay:int,waiter_max_attempts:int,wait_for_completion:bool=False,nodegroup_name:str|None=None,nodegroup_role_arn:str|None=None,create_nodegroup_kwargs:dict|None=None,fargate_profile_name:str|None=None,fargate_pod_execution_role_arn:str|None=None,fargate_selectors:list|None=None,create_fargate_profile_kwargs:dict|None=None,subnets:list[str]|None=None,):log=logging.getLogger(__name__)eks_hook=EksHook(aws_conn_id=aws_conn_id,region_name=region)ifcompute=="nodegroup"andnodegroup_name:# this is to satisfy mypysubnets=subnetsor[]create_nodegroup_kwargs=create_nodegroup_kwargsor{}eks_hook.create_nodegroup(clusterName=cluster_name,nodegroupName=nodegroup_name,subnets=subnets,nodeRole=nodegroup_role_arn,**create_nodegroup_kwargs,)ifwait_for_completion:log.info("Waiting for nodegroup to provision. This will take some time.")wait(waiter=eks_hook.conn.get_waiter("nodegroup_active"),waiter_delay=waiter_delay,waiter_max_attempts=waiter_max_attempts,args={"clusterName":cluster_name,"nodegroupName":nodegroup_name},failure_message="Nodegroup creation failed",status_message="Nodegroup status is",status_args=["nodegroup.status"],)elifcompute=="fargate"andfargate_profile_name:# this is to satisfy mypycreate_fargate_profile_kwargs=create_fargate_profile_kwargsor{}fargate_selectors=fargate_selectorsor[]eks_hook.create_fargate_profile(clusterName=cluster_name,fargateProfileName=fargate_profile_name,podExecutionRoleArn=fargate_pod_execution_role_arn,selectors=fargate_selectors,**create_fargate_profile_kwargs,)ifwait_for_completion:log.info("Waiting for Fargate profile to provision. This will take some time.")wait(waiter=eks_hook.conn.get_waiter("fargate_profile_active"),waiter_delay=waiter_delay,waiter_max_attempts=waiter_max_attempts,args={"clusterName":cluster_name,"fargateProfileName":fargate_profile_name},failure_message="Fargate profile creation failed",status_message="Fargate profile status is",status_args=["fargateProfile.status"],)
[docs]classEksCreateClusterOperator(BaseOperator):""" Creates an Amazon EKS Cluster control plane. Optionally, can also create the supporting compute architecture: - If argument 'compute' is provided with a value of 'nodegroup', will also attempt to create an Amazon EKS Managed Nodegroup for the cluster. See :class:`~airflow.providers.amazon.aws.operators.EksCreateNodegroupOperator` documentation for requirements. - If argument 'compute' is provided with a value of 'fargate', will also attempt to create an AWS Fargate profile for the cluster. See :class:`~airflow.providers.amazon.aws.operators.EksCreateFargateProfileOperator` documentation for requirements. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksCreateClusterOperator` :param cluster_name: The unique name to give to your Amazon EKS Cluster. (templated) :param cluster_role_arn: The Amazon Resource Name (ARN) of the IAM role that provides permissions for the Kubernetes control plane to make calls to AWS API operations on your behalf. (templated) :param resources_vpc_config: The VPC configuration used by the cluster control plane. (templated) :param compute: The type of compute architecture to generate along with the cluster. (templated) Defaults to 'nodegroup' to generate an EKS Managed Nodegroup. :param create_cluster_kwargs: Optional parameters to pass to the CreateCluster API (templated) :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. If compute is assigned the value of 'nodegroup': :param nodegroup_name: *REQUIRED* The unique name to give your Amazon EKS managed node group. (templated) :param nodegroup_role_arn: *REQUIRED* The Amazon Resource Name (ARN) of the IAM role to associate with the Amazon EKS managed node group. (templated) :param create_nodegroup_kwargs: Optional parameters to pass to the CreateNodegroup API (templated) If compute is assigned the value of 'fargate': :param fargate_profile_name: *REQUIRED* The unique name to give your AWS Fargate profile. (templated) :param fargate_pod_execution_role_arn: *REQUIRED* The Amazon Resource Name (ARN) of the pod execution role to use for pods that match the selectors in the AWS Fargate profile. (templated) :param fargate_selectors: The selectors to match for pods to use this AWS Fargate profile. (templated) :param create_fargate_profile_kwargs: Optional parameters to pass to the CreateFargateProfile API (templated) :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check cluster state :param waiter_max_attempts: The maximum number of attempts to check cluster state :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) """
@property@deprecated(reason=("`eks_hook` property is deprecated and will be removed in the future. ""Please use `hook` property instead."),category=AirflowProviderDeprecationWarning,)
[docs]defexecute(self,context:Context):ifself.compute:ifself.computenotinSUPPORTED_COMPUTE_VALUES:raiseValueError("Provided compute type is not supported.")elif(self.compute=="nodegroup")andnotself.nodegroup_role_arn:raiseValueError(MISSING_ARN_MSG.format(compute=NODEGROUP_FULL_NAME,requirement="nodegroup_role_arn"))elif(self.compute=="fargate")andnotself.fargate_pod_execution_role_arn:raiseValueError(MISSING_ARN_MSG.format(compute=FARGATE_FULL_NAME,requirement="fargate_pod_execution_role_arn"))self.hook.create_cluster(name=self.cluster_name,roleArn=self.cluster_role_arn,resourcesVpcConfig=self.resources_vpc_config,**self.create_cluster_kwargs,)# Short circuit early if we don't need to wait to attach compute# and the caller hasn't requested to wait for the cluster either.ifnotany([self.compute,self.wait_for_completion,self.deferrable]):returnNoneself.log.info("Waiting for EKS Cluster to provision. This will take some time.")client=self.hook.connifself.deferrable:self.defer(trigger=EksCreateClusterTrigger(cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region_name=self.region,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,),method_name="deferrable_create_cluster_next",timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay),)try:client.get_waiter("cluster_active").wait(name=self.cluster_name,WaiterConfig={"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts},)except(ClientError,WaiterError)ase:self.log.error("Cluster failed to start and will be torn down.\n%s",e)self.hook.delete_cluster(name=self.cluster_name)client.get_waiter("cluster_deleted").wait(name=self.cluster_name,WaiterConfig={"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts},)raise_create_compute(compute=self.compute,cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region=self.region,wait_for_completion=self.wait_for_completion,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,nodegroup_name=self.nodegroup_name,nodegroup_role_arn=self.nodegroup_role_arn,create_nodegroup_kwargs=self.create_nodegroup_kwargs,fargate_profile_name=self.fargate_profile_name,fargate_pod_execution_role_arn=self.fargate_pod_execution_role_arn,fargate_selectors=self.fargate_selectors,create_fargate_profile_kwargs=self.create_fargate_profile_kwargs,subnets=cast(List[str],self.resources_vpc_config.get("subnetIds")),)
[docs]defdeferrable_create_cluster_next(self,context:Context,event:dict[str,Any]|None=None)->None:ifeventisNone:self.log.error("Trigger error: event is None")raiseAirflowException("Trigger error: event is None")elifevent["status"]=="failed":self.log.error("Cluster failed to start and will be torn down.")self.hook.delete_cluster(name=self.cluster_name)self.defer(trigger=EksDeleteClusterTrigger(cluster_name=self.cluster_name,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region_name=self.region,force_delete_compute=False,),method_name="execute_failed",timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay),)elifevent["status"]=="success":self.log.info("Cluster is ready to provision compute.")_create_compute(compute=self.compute,cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region=self.region,wait_for_completion=self.wait_for_completion,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,nodegroup_name=self.nodegroup_name,nodegroup_role_arn=self.nodegroup_role_arn,create_nodegroup_kwargs=self.create_nodegroup_kwargs,fargate_profile_name=self.fargate_profile_name,fargate_pod_execution_role_arn=self.fargate_pod_execution_role_arn,fargate_selectors=self.fargate_selectors,create_fargate_profile_kwargs=self.create_fargate_profile_kwargs,subnets=cast(List[str],self.resources_vpc_config.get("subnetIds")),)ifself.compute=="fargate":self.defer(trigger=EksCreateFargateProfileTrigger(cluster_name=self.cluster_name,fargate_profile_name=self.fargate_profile_name,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region=self.region,),method_name="execute_complete",timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay),)elifself.compute=="nodegroup":self.defer(trigger=EksCreateNodegroupTrigger(nodegroup_name=self.nodegroup_name,cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region_name=self.region,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,),method_name="execute_complete",timeout=timedelta(seconds=self.waiter_max_attempts*self.waiter_delay),)
[docs]defexecute_failed(self,context:Context,event:dict[str,Any]|None=None)->None:ifeventisNone:self.log.info("Trigger error: event is None")raiseAirflowException("Trigger error: event is None")elifevent["status"]=="deleted":self.log.info("Cluster deleted")raiseAirflowException("Error creating cluster")
[docs]defexecute_complete(self,context:Context,event:dict[str,Any]|None=None)->None:event=validate_execute_complete_event(event)resource="fargate profile"ifself.compute=="fargate"elseself.computeifevent["status"]!="success":raiseAirflowException(f"Error creating {resource}: {event}")self.log.info("%s created successfully",resource)
[docs]classEksCreateNodegroupOperator(BaseOperator):""" Creates an Amazon EKS managed node group for an existing Amazon EKS Cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksCreateNodegroupOperator` :param cluster_name: The name of the Amazon EKS Cluster to create the managed nodegroup in. (templated) :param nodegroup_name: The unique name to give your managed nodegroup. (templated) :param nodegroup_subnets: The subnets to use for the Auto Scaling group that is created for the managed nodegroup. (templated) :param nodegroup_role_arn: The Amazon Resource Name (ARN) of the IAM role to associate with the managed nodegroup. (templated) :param create_nodegroup_kwargs: Optional parameters to pass to the Create Nodegroup API (templated) :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check nodegroup state :param waiter_max_attempts: The maximum number of attempts to check nodegroup state :param deferrable: If True, the operator will wait asynchronously for the nodegroup to be created. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):self.log.info(self.task_id)ifisinstance(self.nodegroup_subnets,str):nodegroup_subnets_list:list[str]=[]ifself.nodegroup_subnets!="":try:nodegroup_subnets_list=cast(list,literal_eval(self.nodegroup_subnets))exceptValueError:self.log.warning("The nodegroup_subnets should be List or string representing ""Python list and is %s. Defaulting to []",self.nodegroup_subnets,)self.nodegroup_subnets=nodegroup_subnets_list_create_compute(compute=self.compute,cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region=self.region,wait_for_completion=self.wait_for_completion,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,nodegroup_name=self.nodegroup_name,nodegroup_role_arn=self.nodegroup_role_arn,create_nodegroup_kwargs=self.create_nodegroup_kwargs,subnets=self.nodegroup_subnets,)ifself.deferrable:self.defer(trigger=EksCreateNodegroupTrigger(cluster_name=self.cluster_name,nodegroup_name=self.nodegroup_name,aws_conn_id=self.aws_conn_id,region_name=self.region,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,),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),)
[docs]classEksCreateFargateProfileOperator(BaseOperator):""" Creates an AWS Fargate profile for an Amazon EKS cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksCreateFargateProfileOperator` :param cluster_name: The name of the Amazon EKS cluster to apply the AWS Fargate profile to. (templated) :param pod_execution_role_arn: The Amazon Resource Name (ARN) of the pod execution role to use for pods that match the selectors in the AWS Fargate profile. (templated) :param selectors: The selectors to match for pods to use this AWS Fargate profile. (templated) :param fargate_profile_name: The unique name to give your AWS Fargate profile. (templated) :param create_fargate_profile_kwargs: Optional parameters to pass to the CreateFargate Profile API (templated) :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check profile status :param waiter_max_attempts: The maximum number of attempts to check the status of the profile. :param deferrable: If True, the operator will wait asynchronously for the profile to be created. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):_create_compute(compute=self.compute,cluster_name=self.cluster_name,aws_conn_id=self.aws_conn_id,region=self.region,wait_for_completion=self.wait_for_completion,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,fargate_profile_name=self.fargate_profile_name,fargate_pod_execution_role_arn=self.pod_execution_role_arn,fargate_selectors=self.selectors,create_fargate_profile_kwargs=self.create_fargate_profile_kwargs,)ifself.deferrable:self.defer(trigger=EksCreateFargateProfileTrigger(cluster_name=self.cluster_name,fargate_profile_name=self.fargate_profile_name,aws_conn_id=self.aws_conn_id,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,region=self.region,),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)),)
[docs]defexecute_complete(self,context:Context,event:dict[str,Any]|None=None)->None:event=validate_execute_complete_event(event)ifevent["status"]!="success":raiseAirflowException(f"Error creating Fargate profile: {event}")self.log.info("Fargate profile created successfully")
[docs]classEksDeleteClusterOperator(BaseOperator):""" Deletes the Amazon EKS Cluster control plane and all nodegroups attached to it. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksDeleteClusterOperator` :param cluster_name: The name of the Amazon EKS Cluster to delete. (templated) :param force_delete_compute: If True, will delete any attached resources. (templated) Defaults to False. :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check cluster state :param waiter_max_attempts: The maximum number of attempts to check cluster state :param deferrable: If True, the operator will wait asynchronously for the cluster to be deleted. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):eks_hook=EksHook(aws_conn_id=self.aws_conn_id,region_name=self.region,)ifself.deferrable:self.defer(trigger=EksDeleteClusterTrigger(cluster_name=self.cluster_name,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,aws_conn_id=self.aws_conn_id,region_name=self.region,force_delete_compute=self.force_delete_compute,),method_name="execute_complete",timeout=timedelta(seconds=self.waiter_delay*self.waiter_max_attempts),)elifself.force_delete_compute:self.delete_any_nodegroups(eks_hook)self.delete_any_fargate_profiles(eks_hook)eks_hook.delete_cluster(name=self.cluster_name)ifself.wait_for_completion:self.log.info("Waiting for cluster to delete. This will take some time.")eks_hook.conn.get_waiter("cluster_deleted").wait(name=self.cluster_name)
[docs]defdelete_any_nodegroups(self,eks_hook)->None:""" Delete all Amazon EKS managed node groups for a provided Amazon EKS Cluster. Amazon EKS managed node groups can be deleted in parallel, so we can send all delete commands in bulk and move on once the count of nodegroups is zero. """nodegroups=eks_hook.list_nodegroups(clusterName=self.cluster_name)ifnodegroups:self.log.info(CAN_NOT_DELETE_MSG.format(compute=NODEGROUP_FULL_NAME,count=len(nodegroups)))forgroupinnodegroups:eks_hook.delete_nodegroup(clusterName=self.cluster_name,nodegroupName=group)# Note this is a custom waiter so we're using hook.get_waiter(), not hook.conn.get_waiter().self.log.info("Waiting for all nodegroups to delete. This will take some time.")eks_hook.get_waiter("all_nodegroups_deleted").wait(clusterName=self.cluster_name)self.log.info(SUCCESS_MSG.format(compute=NODEGROUP_FULL_NAME))
[docs]defdelete_any_fargate_profiles(self,eks_hook)->None:""" Delete all EKS Fargate profiles for a provided Amazon EKS Cluster. EKS Fargate profiles must be deleted one at a time, so we must wait for one to be deleted before sending the next delete command. """fargate_profiles=eks_hook.list_fargate_profiles(clusterName=self.cluster_name)iffargate_profiles:self.log.info(CAN_NOT_DELETE_MSG.format(compute=FARGATE_FULL_NAME,count=len(fargate_profiles)))self.log.info("Waiting for Fargate profiles to delete. This will take some time.")forprofileinfargate_profiles:# The API will return a (cluster) ResourceInUseException if you try# to delete Fargate profiles in parallel the way we can with nodegroups,# so each must be deleted sequentiallyeks_hook.delete_fargate_profile(clusterName=self.cluster_name,fargateProfileName=profile)eks_hook.conn.get_waiter("fargate_profile_deleted").wait(clusterName=self.cluster_name,fargateProfileName=profile)self.log.info(SUCCESS_MSG.format(compute=FARGATE_FULL_NAME))
[docs]classEksDeleteNodegroupOperator(BaseOperator):""" Deletes an Amazon EKS managed node group from an Amazon EKS Cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksDeleteNodegroupOperator` :param cluster_name: The name of the Amazon EKS Cluster associated with your nodegroup. (templated) :param nodegroup_name: The name of the nodegroup to delete. (templated) :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check nodegroup state :param waiter_max_attempts: The maximum number of attempts to check nodegroup state :param deferrable: If True, the operator will wait asynchronously for the nodegroup to be deleted. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):eks_hook=EksHook(aws_conn_id=self.aws_conn_id,region_name=self.region,)eks_hook.delete_nodegroup(clusterName=self.cluster_name,nodegroupName=self.nodegroup_name)ifself.deferrable:self.defer(trigger=EksDeleteNodegroupTrigger(cluster_name=self.cluster_name,nodegroup_name=self.nodegroup_name,aws_conn_id=self.aws_conn_id,region_name=self.region,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,),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),)elifself.wait_for_completion:self.log.info("Waiting for nodegroup to delete. This will take some time.")eks_hook.conn.get_waiter("nodegroup_deleted").wait(clusterName=self.cluster_name,nodegroupName=self.nodegroup_name)
[docs]classEksDeleteFargateProfileOperator(BaseOperator):""" Deletes an AWS Fargate profile from an Amazon EKS Cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksDeleteFargateProfileOperator` :param cluster_name: The name of the Amazon EKS cluster associated with your Fargate profile. (templated) :param fargate_profile_name: The name of the AWS Fargate profile to delete. (templated) :param wait_for_completion: If True, waits for operator to complete. (default: False) (templated) :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param waiter_delay: Time (in seconds) to wait between two consecutive calls to check profile status :param waiter_max_attempts: The maximum number of attempts to check the status of the profile. :param deferrable: If True, the operator will wait asynchronously for the profile to be deleted. This implies waiting for completion. This mode requires aiobotocore module to be installed. (default: False) """
[docs]defexecute(self,context:Context):eks_hook=EksHook(aws_conn_id=self.aws_conn_id,region_name=self.region,)eks_hook.delete_fargate_profile(clusterName=self.cluster_name,fargateProfileName=self.fargate_profile_name)ifself.deferrable:self.defer(trigger=EksDeleteFargateProfileTrigger(cluster_name=self.cluster_name,fargate_profile_name=self.fargate_profile_name,aws_conn_id=self.aws_conn_id,waiter_delay=self.waiter_delay,waiter_max_attempts=self.waiter_max_attempts,region=self.region,),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)),)elifself.wait_for_completion:self.log.info("Waiting for Fargate profile to delete. This will take some time.")eks_hook.conn.get_waiter("fargate_profile_deleted").wait(clusterName=self.cluster_name,fargateProfileName=self.fargate_profile_name,WaiterConfig={"Delay":self.waiter_delay,"MaxAttempts":self.waiter_max_attempts},)
[docs]classEksPodOperator(KubernetesPodOperator):""" Executes a task in a Kubernetes pod on the specified Amazon EKS Cluster. .. seealso:: For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:EksPodOperator` :param cluster_name: The name of the Amazon EKS Cluster to execute the task on. (templated) :param in_cluster: If True, look for config inside the cluster; if False look for a local file path. :param namespace: The namespace in which to execute the pod. (templated) :param pod_name: The unique name to give the pod. (templated) :param aws_profile: The named profile containing the credentials for the AWS CLI tool to use. :param region: Which AWS region the connection should use. (templated) If this is None or empty then the default boto3 behaviour is used. :param aws_conn_id: The Airflow connection used for AWS credentials. (templated) If this is None or empty then the default boto3 behaviour is used. If running Airflow in a distributed manner and aws_conn_id is None or empty, then the default boto3 configuration would be used (and must be maintained on each worker node). :param on_finish_action: What to do when the pod reaches its final state, or the execution is interrupted. If "delete_pod", the pod will be deleted regardless its state; if "delete_succeeded_pod", only succeeded pod will be deleted. You can set to "keep_pod" to keep the pod. Current default is `keep_pod`, but this will be changed in the next major release of this provider. :param is_delete_operator_pod: What to do when the pod reaches its final state, or the execution is interrupted. If True, delete the pod; if False, leave the pod. Current default is False, but this will be changed in the next major release of this provider. Deprecated - use `on_finish_action` instead. """
def__init__(self,cluster_name:str,# Setting in_cluster to False tells the pod that the config# file is stored locally in the worker and not in the cluster.in_cluster:bool=False,namespace:str=DEFAULT_NAMESPACE_NAME,pod_context:str|None=None,pod_name:str|None=None,pod_username:str|None=None,aws_conn_id:str|None=DEFAULT_CONN_ID,region:str|None=None,on_finish_action:str|None=None,is_delete_operator_pod:bool|None=None,**kwargs,)->None:ifis_delete_operator_podisnotNone:warnings.warn("`is_delete_operator_pod` parameter is deprecated, please use `on_finish_action`",AirflowProviderDeprecationWarning,stacklevel=2,)kwargs["on_finish_action"]=(OnFinishAction.DELETE_PODifis_delete_operator_podelseOnFinishAction.KEEP_POD)else:ifon_finish_actionisnotNone:kwargs["on_finish_action"]=OnFinishAction(on_finish_action)else:warnings.warn(f"You have not set parameter `on_finish_action` in class {self.__class__.__name__}. ""Currently the default for this parameter is `keep_pod` but in a future release"" the default will be changed to `delete_pod`. To ensure pods are not deleted in"" the future you will need to set `on_finish_action=keep_pod` explicitly.",AirflowProviderDeprecationWarning,stacklevel=2,)kwargs["on_finish_action"]=OnFinishAction.KEEP_PODself.cluster_name=cluster_nameself.in_cluster=in_clusterself.namespace=namespaceself.pod_name=pod_nameself.aws_conn_id=aws_conn_idself.region=regionsuper().__init__(in_cluster=self.in_cluster,namespace=self.namespace,name=self.pod_name,**kwargs,)# There is no need to manage the kube_config file, as it will be generated automatically.# All Kubernetes parameters (except config_file) are also valid for the EksPodOperator.ifself.config_file:raiseAirflowException("The config_file is not an allowed parameter for the EksPodOperator.")