## 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."""Base operator for all operators."""importabcimportcopyimportfunctoolsimportloggingimportsysimportwarningsfromabcimportABCMeta,abstractmethodfromdatetimeimportdatetime,timedeltafrominspectimportsignaturefromtypingimport(TYPE_CHECKING,Any,Callable,ClassVar,Dict,FrozenSet,Iterable,List,Optional,Sequence,Set,Tuple,Type,TypeVar,Union,cast,)importattrimportjinja2fromdateutil.relativedeltaimportrelativedeltafromsqlalchemy.ormimportSessionfromsqlalchemy.orm.excimportNoResultFoundimportairflow.templatesfromairflow.compat.functoolsimportcached_propertyfromairflow.configurationimportconffromairflow.exceptionsimportAirflowException,TaskDeferredfromairflow.lineageimportapply_lineage,prepare_lineagefromairflow.models.baseimportOperatorfromairflow.models.paramimportParamsDictfromairflow.models.poolimportPoolfromairflow.models.taskinstanceimportContext,TaskInstance,clear_task_instancesfromairflow.models.taskmixinimportTaskMixinfromairflow.models.xcomimportXCOM_RETURN_KEYfromairflow.ti_deps.deps.base_ti_depimportBaseTIDepfromairflow.ti_deps.deps.not_in_retry_period_depimportNotInRetryPeriodDepfromairflow.ti_deps.deps.not_previously_skipped_depimportNotPreviouslySkippedDepfromairflow.ti_deps.deps.prev_dagrun_depimportPrevDagrunDepfromairflow.ti_deps.deps.trigger_rule_depimportTriggerRuleDepfromairflow.triggers.baseimportBaseTriggerfromairflow.utilsimporttimezonefromairflow.utils.edgemodifierimportEdgeModifierfromairflow.utils.helpersimportrender_template_as_native,render_template_to_string,validate_keyfromairflow.utils.log.logging_mixinimportLoggingMixinfromairflow.utils.operator_resourcesimportResourcesfromairflow.utils.sessionimportprovide_sessionfromairflow.utils.trigger_ruleimportTriggerRulefromairflow.utils.weight_ruleimportWeightRuleifTYPE_CHECKING:fromairflow.models.xcom_argimportXComArgfromairflow.utils.task_groupimportTaskGroup
[docs]classBaseOperatorMeta(abc.ABCMeta):"""Metaclass of BaseOperator."""@classmethoddef_apply_defaults(cls,func:T)->T:""" Function decorator that Looks for an argument named "default_args", and fills the unspecified arguments from it. Since python2.* isn't clear about which arguments are missing when calling a function, and that this can be quite confusing with multi-level inheritance and argument defaults, this decorator also alerts with specific information about the missing arguments. """# Cache inspect.signature for the wrapper closure to avoid calling it# at every decorated invocation. This is separate sig_cache created# per decoration, i.e. each function decorated using apply_defaults will# have a different sig_cache.sig_cache=signature(func)non_optional_args={namefor(name,param)insig_cache.parameters.items()ifparam.default==param.emptyandparam.name!='self'andparam.kindnotin(param.VAR_POSITIONAL,param.VAR_KEYWORD)}classautostacklevel_warn:def__init__(self):self.warnings=__import__('warnings')def__getattr__(self,name):returngetattr(self.warnings,name)def__dir__(self):returndir(self.warnings)defwarn(self,message,category=None,stacklevel=1,source=None):self.warnings.warn(message,category,stacklevel+2,source)iffunc.__globals__.get('warnings')issys.modules['warnings']:# Yes, this is slightly hacky, but it _automatically_ sets the right# stacklevel parameter to `warnings.warn` to ignore the decorator. Now# that the decorator is applied automatically, this makes the needed# stacklevel parameter less confusing.func.__globals__['warnings']=autostacklevel_warn()@functools.wraps(func)defapply_defaults(self,*args:Any,**kwargs:Any)->Any:fromairflow.models.dagimportDagContextfromairflow.utils.task_groupimportTaskGroupContextiflen(args)>0:raiseAirflowException("Use keyword arguments when initializing operators")dag_args:Dict[str,Any]={}dag_params:Dict[str,Any]={}dag=kwargs.get('dag')orDagContext.get_current_dag()ifdag:dag_args=copy.copy(dag.default_args)or{}dag_params=copy.deepcopy(dag.params)or{}task_group=TaskGroupContext.get_current_task_group(dag)iftask_group:dag_args.update(task_group.default_args)params=kwargs.get('params',{})or{}dag_params.update(params)default_args={}if'default_args'inkwargs:default_args=kwargs['default_args']if'params'indefault_args:dag_params.update(default_args['params'])deldefault_args['params']dag_args.update(default_args)default_args=dag_argsforarginsig_cache.parameters:ifargnotinkwargsandargindefault_args:kwargs[arg]=default_args[arg]missing_args=list(non_optional_args-set(kwargs))ifmissing_args:msg=f"Argument {missing_args} is required"raiseAirflowException(msg)ifdag_params:kwargs['params']=dag_paramsifdefault_args:kwargs['default_args']=default_argsifhasattr(self,'_hook_apply_defaults'):args,kwargs=self._hook_apply_defaults(*args,**kwargs)result=func(self,*args,**kwargs)# Here we set upstream task defined by XComArgs passed to template fields of the operatorself.set_xcomargs_dependencies()# Mark instance as instantiated https://docs.python.org/3/tutorial/classes.html#private-variablesself._BaseOperator__instantiated=Truereturnresultreturncast(T,apply_defaults)
[docs]@functools.total_orderingclassBaseOperator(Operator,LoggingMixin,TaskMixin,metaclass=BaseOperatorMeta):""" Abstract base class for all operators. Since operators create objects that become nodes in the dag, BaseOperator contains many recursive methods for dag crawling behavior. To derive this class, you are expected to override the constructor as well as the 'execute' method. Operators derived from this class should perform or trigger certain tasks synchronously (wait for completion). Example of operators could be an operator that runs a Pig job (PigOperator), a sensor operator that waits for a partition to land in Hive (HiveSensorOperator), or one that moves data from Hive to MySQL (Hive2MySqlOperator). Instances of these operators (tasks) target specific operations, running specific scripts, functions or data transfers. This class is abstract and shouldn't be instantiated. Instantiating a class derived from this one results in the creation of a task object, which ultimately becomes a node in DAG objects. Task dependencies should be set by using the set_upstream and/or set_downstream methods. :param task_id: a unique, meaningful id for the task :type task_id: str :param owner: the owner of the task. Using a meaningful description (e.g. user/person/team/role name) to clarify ownership is recommended. :type owner: str :param email: the 'to' email address(es) used in email alerts. This can be a single email or multiple ones. Multiple addresses can be specified as a comma or semi-colon separated string or by passing a list of strings. :type email: str or list[str] :param email_on_retry: Indicates whether email alerts should be sent when a task is retried :type email_on_retry: bool :param email_on_failure: Indicates whether email alerts should be sent when a task failed :type email_on_failure: bool :param retries: the number of retries that should be performed before failing the task :type retries: int :param retry_delay: delay between retries :type retry_delay: datetime.timedelta :param retry_exponential_backoff: allow progressive longer waits between retries by using exponential backoff algorithm on retry delay (delay will be converted into seconds) :type retry_exponential_backoff: bool :param max_retry_delay: maximum delay interval between retries :type max_retry_delay: datetime.timedelta :param start_date: The ``start_date`` for the task, determines the ``execution_date`` for the first task instance. The best practice is to have the start_date rounded to your DAG's ``schedule_interval``. Daily jobs have their start_date some day at 00:00:00, hourly jobs have their start_date at 00:00 of a specific hour. Note that Airflow simply looks at the latest ``execution_date`` and adds the ``schedule_interval`` to determine the next ``execution_date``. It is also very important to note that different tasks' dependencies need to line up in time. If task A depends on task B and their start_date are offset in a way that their execution_date don't line up, A's dependencies will never be met. If you are looking to delay a task, for example running a daily task at 2AM, look into the ``TimeSensor`` and ``TimeDeltaSensor``. We advise against using dynamic ``start_date`` and recommend using fixed ones. Read the FAQ entry about start_date for more information. :type start_date: datetime.datetime :param end_date: if specified, the scheduler won't go beyond this date :type end_date: datetime.datetime :param depends_on_past: when set to true, task instances will run sequentially and only if the previous instance has succeeded or has been skipped. The task instance for the start_date is allowed to run. :type depends_on_past: bool :param wait_for_downstream: when set to true, an instance of task X will wait for tasks immediately downstream of the previous instance of task X to finish successfully or be skipped before it runs. This is useful if the different instances of a task X alter the same asset, and this asset is used by tasks downstream of task X. Note that depends_on_past is forced to True wherever wait_for_downstream is used. Also note that only tasks *immediately* downstream of the previous task instance are waited for; the statuses of any tasks further downstream are ignored. :type wait_for_downstream: bool :param dag: a reference to the dag the task is attached to (if any) :type dag: airflow.models.DAG :param priority_weight: priority weight of this task against other task. This allows the executor to trigger higher priority tasks before others when things get backed up. Set priority_weight as a higher number for more important tasks. :type priority_weight: int :param weight_rule: weighting method used for the effective total priority weight of the task. Options are: ``{ downstream | upstream | absolute }`` default is ``downstream`` When set to ``downstream`` the effective weight of the task is the aggregate sum of all downstream descendants. As a result, upstream tasks will have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and desire to have all upstream tasks to complete for all runs before each dag can continue processing downstream tasks. When set to ``upstream`` the effective weight is the aggregate sum of all upstream ancestors. This is the opposite where downstream tasks have higher weight and will be scheduled more aggressively when using positive weight values. This is useful when you have multiple dag run instances and prefer to have each dag complete before starting upstream tasks of other dags. When set to ``absolute``, the effective weight is the exact ``priority_weight`` specified without additional weighting. You may want to do this when you know exactly what priority weight each task should have. Additionally, when set to ``absolute``, there is bonus effect of significantly speeding up the task creation process as for very large DAGs. Options can be set as string or using the constants defined in the static class ``airflow.utils.WeightRule`` :type weight_rule: str :param queue: which queue to target when running this job. Not all executors implement queue management, the CeleryExecutor does support targeting specific queues. :type queue: str :param pool: the slot pool this task should run in, slot pools are a way to limit concurrency for certain tasks :type pool: str :param pool_slots: the number of pool slots this task should use (>= 1) Values less than 1 are not allowed. :type pool_slots: int :param sla: time by which the job is expected to succeed. Note that this represents the ``timedelta`` after the period is closed. For example if you set an SLA of 1 hour, the scheduler would send an email soon after 1:00AM on the ``2016-01-02`` if the ``2016-01-01`` instance has not succeeded yet. The scheduler pays special attention for jobs with an SLA and sends alert emails for SLA misses. SLA misses are also recorded in the database for future reference. All tasks that share the same SLA time get bundled in a single email, sent soon after that time. SLA notification are sent once and only once for each task instance. :type sla: datetime.timedelta :param execution_timeout: max time allowed for the execution of this task instance, if it goes beyond it will raise and fail. :type execution_timeout: datetime.timedelta :param on_failure_callback: a function to be called when a task instance of this task fails. a context dictionary is passed as a single parameter to this function. Context contains references to related objects to the task instance and is documented under the macros section of the API. :type on_failure_callback: TaskStateChangeCallback :param on_execute_callback: much like the ``on_failure_callback`` except that it is executed right before the task is executed. :type on_execute_callback: TaskStateChangeCallback :param on_retry_callback: much like the ``on_failure_callback`` except that it is executed when retries occur. :type on_retry_callback: TaskStateChangeCallback :param on_success_callback: much like the ``on_failure_callback`` except that it is executed when the task succeeds. :type on_success_callback: TaskStateChangeCallback :param pre_execute: a function to be called immediately before task execution, receiving a context dictionary; raising an exception will prevent the task from being executed. |experimental| :type pre_execute: TaskPreExecuteHook :param post_execute: a function to be called immediately after task execution, receiving a context dictionary and task result; raising an exception will prevent the task from succeeding. |experimental| :type post_execute: TaskPostExecuteHook :param trigger_rule: defines the rule by which dependencies are applied for the task to get triggered. Options are: ``{ all_success | all_failed | all_done | one_success | one_failed | none_failed | none_failed_min_one_success | none_skipped | always}`` default is ``all_success``. Options can be set as string or using the constants defined in the static class ``airflow.utils.TriggerRule`` :type trigger_rule: str :param resources: A map of resource parameter names (the argument names of the Resources constructor) to their values. :type resources: dict :param run_as_user: unix username to impersonate while running the task :type run_as_user: str :param max_active_tis_per_dag: When set, a task will be able to limit the concurrent runs across execution_dates. :type max_active_tis_per_dag: int :param executor_config: Additional task-level configuration parameters that are interpreted by a specific executor. Parameters are namespaced by the name of executor. **Example**: to run this task in a specific docker container through the KubernetesExecutor :: MyOperator(..., executor_config={ "KubernetesExecutor": {"image": "myCustomDockerImage"} } ) :type executor_config: dict :param do_xcom_push: if True, an XCom is pushed containing the Operator's result :type do_xcom_push: bool :param task_group: The TaskGroup to which the task should belong. This is typically provided when not using a TaskGroup as a context manager. :type task_group: airflow.utils.task_group.TaskGroup :param doc: Add documentation or notes to your Task objects that is visible in Task Instance details View in the Webserver :type doc: str :param doc_md: Add documentation (in Markdown format) or notes to your Task objects that is visible in Task Instance details View in the Webserver :type doc_md: str :param doc_rst: Add documentation (in RST format) or notes to your Task objects that is visible in Task Instance details View in the Webserver :type doc_rst: str :param doc_json: Add documentation (in JSON format) or notes to your Task objects that is visible in Task Instance details View in the Webserver :type doc_json: str :param doc_yaml: Add documentation (in YAML format) or notes to your Task objects that is visible in Task Instance details View in the Webserver :type doc_yaml: str """# For derived classes to define which fields will get jinjaified
# base list which includes all the attrs that don't need deep copy._base_operator_shallow_copy_attrs:Tuple[str,...]=('user_defined_macros','user_defined_filters','params','_log',)# each operator should override this class attr for shallow copy attrs.
# The _serialized_fields are lazily loaded when get_serialized_fields() method is called__serialized_fields:Optional[FrozenSet[str]]=None_comps={'task_id','dag_id','owner','email','email_on_retry','retry_delay','retry_exponential_backoff','max_retry_delay','start_date','depends_on_past','wait_for_downstream','priority_weight','sla','execution_timeout','on_execute_callback','on_failure_callback','on_success_callback','on_retry_callback','do_xcom_push',}# Defines if the operator supports lineage without manual definitions
# If True then the class constructor was called__instantiated=False# Set to True before calling execute method_lock_for_execution=Falsedef__init__(self,task_id:str,owner:str=conf.get('operators','DEFAULT_OWNER'),email:Optional[Union[str,Iterable[str]]]=None,email_on_retry:bool=conf.getboolean('email','default_email_on_retry',fallback=True),email_on_failure:bool=conf.getboolean('email','default_email_on_failure',fallback=True),retries:Optional[int]=conf.getint('core','default_task_retries',fallback=0),retry_delay:timedelta=timedelta(seconds=300),retry_exponential_backoff:bool=False,max_retry_delay:Optional[timedelta]=None,start_date:Optional[datetime]=None,end_date:Optional[datetime]=None,depends_on_past:bool=False,wait_for_downstream:bool=False,dag=None,params:Optional[Dict]=None,default_args:Optional[Dict]=None,priority_weight:int=1,weight_rule:str=conf.get('core','default_task_weight_rule',fallback=WeightRule.DOWNSTREAM),queue:str=conf.get('operators','default_queue'),pool:Optional[str]=None,pool_slots:int=1,sla:Optional[timedelta]=None,execution_timeout:Optional[timedelta]=None,on_execute_callback:Optional[TaskStateChangeCallback]=None,on_failure_callback:Optional[TaskStateChangeCallback]=None,on_success_callback:Optional[TaskStateChangeCallback]=None,on_retry_callback:Optional[TaskStateChangeCallback]=None,pre_execute:Optional[TaskPreExecuteHook]=None,post_execute:Optional[TaskPostExecuteHook]=None,trigger_rule:str=TriggerRule.ALL_SUCCESS,resources:Optional[Dict]=None,run_as_user:Optional[str]=None,task_concurrency:Optional[int]=None,max_active_tis_per_dag:Optional[int]=None,executor_config:Optional[Dict]=None,do_xcom_push:bool=True,inlets:Optional[Any]=None,outlets:Optional[Any]=None,task_group:Optional["TaskGroup"]=None,doc:Optional[str]=None,doc_md:Optional[str]=None,doc_json:Optional[str]=None,doc_yaml:Optional[str]=None,doc_rst:Optional[str]=None,**kwargs,):fromairflow.models.dagimportDagContextfromairflow.utils.task_groupimportTaskGroupContextsuper().__init__()ifkwargs:ifnotconf.getboolean('operators','ALLOW_ILLEGAL_ARGUMENTS'):raiseAirflowException("Invalid arguments were passed to {c} (task_id: {t}). Invalid ""arguments were:\n**kwargs: {k}".format(c=self.__class__.__name__,k=kwargs,t=task_id),)warnings.warn('Invalid arguments were passed to {c} (task_id: {t}). ''Support for passing such arguments will be dropped in ''future. Invalid arguments were:''\n**kwargs: {k}'.format(c=self.__class__.__name__,k=kwargs,t=task_id),category=PendingDeprecationWarning,stacklevel=3,)validate_key(task_id)self.task_id=task_idself.label=task_idtask_group=task_grouporTaskGroupContext.get_current_task_group(dag)iftask_group:self.task_id=task_group.child_id(task_id)task_group.add(self)self.owner=ownerself.email=emailself.email_on_retry=email_on_retryself.email_on_failure=email_on_failureself.start_date=start_dateifstart_dateandnotisinstance(start_date,datetime):self.log.warning("start_date for %s isn't datetime.datetime",self)elifstart_date:self.start_date=timezone.convert_to_utc(start_date)self.end_date=end_dateifend_date:self.end_date=timezone.convert_to_utc(end_date)iftrigger_rule=="dummy":warnings.warn("dummy Trigger Rule is deprecated. Please use `TriggerRule.ALWAYS`.",DeprecationWarning,stacklevel=2,)trigger_rule=TriggerRule.ALWAYSiftrigger_rule=="none_failed_or_skipped":warnings.warn("none_failed_or_skipped Trigger Rule is deprecated. ""Please use `none_failed_min_one_success`.",DeprecationWarning,stacklevel=2,)trigger_rule=TriggerRule.NONE_FAILED_MIN_ONE_SUCCESSifnotTriggerRule.is_valid(trigger_rule):raiseAirflowException("The trigger_rule must be one of {all_triggers},""'{d}.{t}'; received '{tr}'.".format(all_triggers=TriggerRule.all_triggers(),d=dag.dag_idifdagelse"",t=task_id,tr=trigger_rule,))self.trigger_rule=trigger_ruleself.depends_on_past=depends_on_pastself.wait_for_downstream=wait_for_downstreamifwait_for_downstream:self.depends_on_past=TrueifretriesisnotNoneandnotisinstance(retries,int):try:parsed_retries=int(retries)except(TypeError,ValueError):raiseAirflowException(f"'retries' type must be int, not {type(retries).__name__}")self.log.warning("Implicitly converting 'retries' for %s from %r to int",self,retries)retries=parsed_retriesself.retries=retriesself.queue=queueself.pool=Pool.DEFAULT_POOL_NAMEifpoolisNoneelsepoolself.pool_slots=pool_slotsifself.pool_slots<1:raiseAirflowException(f"pool slots for {self.task_id} in dag {dag.dag_id} cannot be less than 1")self.sla=slaself.execution_timeout=execution_timeoutself.on_execute_callback=on_execute_callbackself.on_failure_callback=on_failure_callbackself.on_success_callback=on_success_callbackself.on_retry_callback=on_retry_callbackself._pre_execute_hook=pre_executeself._post_execute_hook=post_executeifisinstance(retry_delay,timedelta):self.retry_delay=retry_delayelse:self.log.debug("Retry_delay isn't timedelta object, assuming secs")self.retry_delay=timedelta(seconds=retry_delay)self.retry_exponential_backoff=retry_exponential_backoffself.max_retry_delay=max_retry_delayifmax_retry_delay:ifisinstance(max_retry_delay,timedelta):self.max_retry_delay=max_retry_delayelse:self.log.debug("max_retry_delay isn't a timedelta object, assuming secs")self.max_retry_delay=timedelta(seconds=max_retry_delay)self.params=ParamsDict(params)ifpriority_weightisnotNoneandnotisinstance(priority_weight,int):raiseAirflowException(f"`priority_weight` for task '{self.task_id}' only accepts integers, "f"received '{type(priority_weight)}'.")self.priority_weight=priority_weightifnotWeightRule.is_valid(weight_rule):raiseAirflowException("The weight_rule must be one of {all_weight_rules},""'{d}.{t}'; received '{tr}'.".format(all_weight_rules=WeightRule.all_weight_rules,d=dag.dag_idifdagelse"",t=task_id,tr=weight_rule,))self.weight_rule=weight_ruleself.resources:Optional[Resources]=Resources(**resources)ifresourceselseNoneself.run_as_user=run_as_useriftask_concurrencyandnotmax_active_tis_per_dag:# TODO: Remove in Airflow 3.0warnings.warn("The 'task_concurrency' parameter is deprecated. Please use 'max_active_tis_per_dag'.",DeprecationWarning,stacklevel=2,)max_active_tis_per_dag=task_concurrencyself.max_active_tis_per_dag=max_active_tis_per_dagself.executor_config=executor_configor{}self.do_xcom_push=do_xcom_pushself.doc_md=doc_mdself.doc_json=doc_jsonself.doc_yaml=doc_yamlself.doc_rst=doc_rstself.doc=doc# Private attributesself._upstream_task_ids:Set[str]=set()self._downstream_task_ids:Set[str]=set()self._dag=Noneself.dag=dagorDagContext.get_current_dag()# subdag parameter is only set for SubDagOperator.# Setting it to None by default as other Operators do not have that fieldfromairflow.models.dagimportDAGself.subdag:Optional[DAG]=Noneself._log=logging.getLogger("airflow.task.operators")# Lineageself.inlets:List=[]self.outlets:List=[]self._inlets:List=[]self._outlets:List=[]ifinlets:self._inlets=(inletsifisinstance(inlets,list)else[inlets,])ifoutlets:self._outlets=(outletsifisinstance(outlets,list)else[outlets,])
[docs]def__eq__(self,other):iftype(self)istype(other):# Use getattr() instead of __dict__ as __dict__ doesn't return# correct values for properties.returnall(getattr(self,c,None)==getattr(other,c,None)forcinself._comps)returnFalse
[docs]def__or__(self,other):""" Called for [This Operator] | [Operator], The inlets of other will be set to pickup the outlets from this operator. Other will be set as a downstream task of this operator. """ifisinstance(other,BaseOperator):ifnotself._outletsandnotself.supports_lineage:raiseValueError("No outlets defined for this operator")other.add_inlets([self.task_id])self.set_downstream(other)else:raiseTypeError(f"Right hand side ({other}) is not an Operator")returnself
[docs]def__gt__(self,other):""" Called for [Operator] > [Outlet], so that if other is an attr annotated object it is set as an outlet of this Operator. """ifnotisinstance(other,Iterable):other=[other]forobjinother:ifnotattr.has(obj):raiseTypeError(f"Left hand side ({obj}) is not an outlet")self.add_outlets(other)returnself
[docs]def__lt__(self,other):""" Called for [Inlet] > [Operator] or [Operator] < [Inlet], so that if other is an attr annotated object it is set as an inlet to this operator """ifnotisinstance(other,Iterable):other=[other]forobjinother:ifnotattr.has(obj):raiseTypeError(f"{obj} cannot be an inlet")self.add_inlets(other)returnself
[docs]def__setattr__(self,key,value):super().__setattr__(key,value)ifself._lock_for_execution:# Skip any custom behaviour during executereturnifself.__instantiatedandkeyinself.template_fields:# Resolve upstreams set by assigning an XComArg after initializing# an operator, example:# op = BashOperator()# op.bash_command = "sleep 1"self.set_xcomargs_dependencies()
[docs]defadd_inlets(self,inlets:Iterable[Any]):"""Sets inlets to this operator"""self._inlets.extend(inlets)
[docs]defadd_outlets(self,outlets:Iterable[Any]):"""Defines the outlets of this operator"""self._outlets.extend(outlets)
[docs]defget_inlet_defs(self):""":return: list of inlets defined for this operator"""returnself._inlets
[docs]defget_outlet_defs(self):""":return: list of outlets defined for this operator"""returnself._outlets
@property
[docs]defdag(self)->Any:"""Returns the Operator's DAG if set, otherwise raises an error"""ifself.has_dag():returnself._dagelse:raiseAirflowException(f'Operator {self} has not been assigned to a DAG yet')
@dag.setterdefdag(self,dag:Any):""" Operators can be assigned to one DAG, one time. Repeat assignments to that same DAG are ok. """fromairflow.models.dagimportDAGifdagisNone:self._dag=Nonereturnifnotisinstance(dag,DAG):raiseTypeError(f'Expected DAG; received {dag.__class__.__name__}')elifself.has_dag()andself.dagisnotdag:raiseAirflowException(f"The DAG assigned to {self} can not be changed.")elifself.task_idnotindag.task_dict:dag.add_task(self)elifself.task_idindag.task_dictanddag.task_dict[self.task_id]isnotself:dag.add_task(self)self._dag=dag
[docs]defhas_dag(self):"""Returns True if the Operator has been assigned to a DAG."""returngetattr(self,'_dag',None)isnotNone
@property
[docs]defdag_id(self)->str:"""Returns dag id if it has one or an adhoc + owner"""ifself.has_dag():returnself.dag.dag_idelse:return'adhoc_'+self.owner
})""" Returns the set of dependencies for the operator. These differ from execution context dependencies in that they are specific to tasks and can be extended/overridden by subclasses. """
[docs]defprepare_for_execution(self)->"BaseOperator":""" Lock task for execution to disable custom action in __setattr__ and returns a copy of the task """other=copy.copy(self)other._lock_for_execution=Truereturnother
[docs]defset_xcomargs_dependencies(self)->None:""" Resolves upstream dependencies of a task. In this way passing an ``XComArg`` as value for a template field will result in creating upstream relation between two tasks. **Example**: :: with DAG(...): generate_content = GenerateContentOperator(task_id="generate_content") send_email = EmailOperator(..., html_content=generate_content.output) # This is equivalent to with DAG(...): generate_content = GenerateContentOperator(task_id="generate_content") send_email = EmailOperator( ..., html_content="{{ task_instance.xcom_pull('generate_content') }}" ) generate_content >> send_email """fromairflow.models.xcom_argimportXComArgdefapply_set_upstream(arg:Any):ifisinstance(arg,XComArg):self.set_upstream(arg.operator)elifisinstance(arg,(tuple,set,list)):foreleminarg:apply_set_upstream(elem)elifisinstance(arg,dict):foreleminarg.values():apply_set_upstream(elem)elifhasattr(arg,"template_fields"):foreleminarg.template_fields:apply_set_upstream(elem)forfieldinself.template_fields:ifhasattr(self,field):arg=getattr(self,field)apply_set_upstream(arg)
@property
[docs]defpriority_weight_total(self)->int:""" Total priority weight for the task. It might include all upstream or downstream tasks. depending on the weight rule. - WeightRule.ABSOLUTE - only own weight - WeightRule.DOWNSTREAM - adds priority weight of all downstream tasks - WeightRule.UPSTREAM - adds priority weight of all upstream tasks """ifself.weight_rule==WeightRule.ABSOLUTE:returnself.priority_weightelifself.weight_rule==WeightRule.DOWNSTREAM:upstream=Falseelifself.weight_rule==WeightRule.UPSTREAM:upstream=Trueelse:upstream=Falseifnotself._dag:returnself.priority_weightfromairflow.models.dagimportDAGdag:DAG=self._dagreturnself.priority_weight+sum(map(lambdatask_id:dag.task_dict[task_id].priority_weight,self.get_flat_relative_ids(upstream=upstream),
))@cached_property
[docs]defoperator_extra_link_dict(self)->Dict[str,Any]:"""Returns dictionary of all extra links for the operator"""op_extra_links_from_plugin:Dict[str,Any]={}fromairflowimportplugins_managerplugins_manager.initialize_extra_operators_links_plugins()ifplugins_manager.operator_extra_linksisNone:raiseAirflowException("Can't load operators")foropeinplugins_manager.operator_extra_links:ifope.operatorsandself.__class__inope.operators:op_extra_links_from_plugin.update({ope.name:ope})operator_extra_links_all={link.name:linkforlinkinself.operator_extra_links}# Extra links defined in Plugins overrides operator links defined in operatoroperator_extra_links_all.update(op_extra_links_from_plugin)returnoperator_extra_links_all
@cached_property
[docs]defglobal_operator_extra_link_dict(self)->Dict[str,Any]:"""Returns dictionary of all global extra links"""fromairflowimportplugins_managerplugins_manager.initialize_extra_operators_links_plugins()ifplugins_manager.global_operator_extra_linksisNone:raiseAirflowException("Can't load operators")return{link.name:linkforlinkinplugins_manager.global_operator_extra_links}
@prepare_lineage
[docs]defpre_execute(self,context:Any):"""This hook is triggered right before self.execute() is called."""ifself._pre_execute_hookisnotNone:self._pre_execute_hook(context)
[docs]defexecute(self,context:Any):""" This is the main method to derive when creating an operator. Context is the same dictionary used as when rendering jinja templates. Refer to get_template_context for more context. """raiseNotImplementedError()
@apply_lineage
[docs]defpost_execute(self,context:Any,result:Any=None):""" This hook is triggered right after self.execute() is called. It is passed the execution context and any results returned by the operator. """ifself._post_execute_hookisnotNone:self._post_execute_hook(context,result)
""" Override this method to cleanup subprocesses when a task instance gets killed. Any use of the threading, subprocess or multiprocessing module within an operator needs to be cleaned up or it will leave ghost processes behind. """
[docs]def__deepcopy__(self,memo):""" Hack sorting double chained task lists by task_id to avoid hitting max_depth on deepcopy operations. """sys.setrecursionlimit(5000)# TODO fix this in a better waycls=self.__class__result=cls.__new__(cls)memo[id(self)]=resultshallow_copy=cls.shallow_copy_attrs+cls._base_operator_shallow_copy_attrsfork,vinself.__dict__.items():ifknotinshallow_copy:setattr(result,k,copy.deepcopy(v,memo))else:setattr(result,k,copy.copy(v))returnresult
[docs]defrender_template_fields(self,context:Context,jinja_env:Optional[jinja2.Environment]=None,)->None:""" Template all attributes listed in template_fields. Note this operation is irreversible. :param context: Dict with values to apply on content :type context: dict :param jinja_env: Jinja environment :type jinja_env: jinja2.Environment """ifnotjinja_env:jinja_env=self.get_template_env()self._do_render_template_fields(self,self.template_fields,context,jinja_env,set())
[docs]defrender_template(self,content:Any,context:Context,jinja_env:Optional[jinja2.Environment]=None,seen_oids:Optional[Set]=None,)->Any:""" Render a templated string. The content can be a collection holding multiple templated strings and will be templated recursively. :param content: Content to template. Only strings can be templated (may be inside collection). :type content: Any :param context: Dict with values to apply on templated content :type context: dict :param jinja_env: Jinja environment. Can be provided to avoid re-creating Jinja environments during recursion. :type jinja_env: jinja2.Environment :param seen_oids: template fields already rendered (to avoid RecursionError on circular dependencies) :type seen_oids: set :return: Templated content """ifnotjinja_env:jinja_env=self.get_template_env()# Imported here to avoid circular dependencyfromairflow.models.paramimportDagParamfromairflow.models.xcom_argimportXComArgifisinstance(content,str):ifany(content.endswith(ext)forextinself.template_ext):# Content contains a filepath.template=jinja_env.get_template(content)else:template=jinja_env.from_string(content)ifself.has_dag()andself.dag.render_template_as_native_obj:returnrender_template_as_native(template,context)returnrender_template_to_string(template,context)elifisinstance(content,(XComArg,DagParam)):returncontent.resolve(context)ifisinstance(content,tuple):iftype(content)isnottuple:# Special case for named tuplesreturncontent.__class__(*(self.render_template(element,context,jinja_env)forelementincontent))else:returntuple(self.render_template(element,context,jinja_env)forelementincontent)elifisinstance(content,list):return[self.render_template(element,context,jinja_env)forelementincontent]elifisinstance(content,dict):return{key:self.render_template(value,context,jinja_env)forkey,valueincontent.items()}elifisinstance(content,set):return{self.render_template(element,context,jinja_env)forelementincontent}else:ifseen_oidsisNone:seen_oids=set()self._render_nested_template_fields(content,context,jinja_env,seen_oids)returncontent
def_render_nested_template_fields(self,content:Any,context:Context,jinja_env:jinja2.Environment,seen_oids:Set)->None:ifid(content)notinseen_oids:seen_oids.add(id(content))try:nested_template_fields=content.template_fieldsexceptAttributeError:# content has no inner template fieldsreturnself._do_render_template_fields(content,nested_template_fields,context,jinja_env,seen_oids)
[docs]defget_template_env(self)->jinja2.Environment:"""Fetch a Jinja template environment from the DAG or instantiate empty environment if no DAG."""return(self.dag.get_template_env()ifself.has_dag()elseairflow.templates.SandboxedEnvironment(cache_size=0)
""" Hook that is triggered after the templated fields get replaced by their content. If you need your operator to alter the content of the file before the template is rendered, it should override this method to do so. """
[docs]defresolve_template_files(self)->None:"""Getting the content of files for template_field / template_ext"""ifself.template_ext:forfieldinself.template_fields:content=getattr(self,field,None)ifcontentisNone:continueelifisinstance(content,str)andany(content.endswith(ext)forextinself.template_ext):env=self.get_template_env()try:setattr(self,field,env.loader.get_source(env,content)[0])exceptExceptionase:self.log.exception(e)elifisinstance(content,list):env=self.dag.get_template_env()fori,iteminenumerate(content):ifisinstance(item,str)andany(item.endswith(ext)forextinself.template_ext):try:content[i]=env.loader.get_source(env,item)[0]exceptExceptionase:self.log.exception(e)self.prepare_template()
@property
[docs]defupstream_list(self)->List["BaseOperator"]:"""@property: list of tasks directly upstream"""return[self.dag.get_task(tid)fortidinself._upstream_task_ids]
@property
[docs]defupstream_task_ids(self)->Set[str]:"""@property: set of ids of tasks directly upstream"""returnself._upstream_task_ids
@property
[docs]defdownstream_list(self)->List["BaseOperator"]:"""@property: list of tasks directly downstream"""return[self.dag.get_task(tid)fortidinself._downstream_task_ids]
@property
[docs]defdownstream_task_ids(self)->Set[str]:"""@property: set of ids of tasks directly downstream"""returnself._downstream_task_ids
@provide_session
[docs]defclear(self,start_date:Optional[datetime]=None,end_date:Optional[datetime]=None,upstream:bool=False,downstream:bool=False,session:Session=None,):""" Clears the state of task instances associated with the task, following the parameters specified. """qry=session.query(TaskInstance).filter(TaskInstance.dag_id==self.dag_id)ifstart_date:qry=qry.filter(TaskInstance.execution_date>=start_date)ifend_date:qry=qry.filter(TaskInstance.execution_date<=end_date)tasks=[self.task_id]ifupstream:tasks+=[t.task_idfortinself.get_flat_relatives(upstream=True)]ifdownstream:tasks+=[t.task_idfortinself.get_flat_relatives(upstream=False)]qry=qry.filter(TaskInstance.task_id.in_(tasks))results=qry.all()count=len(results)clear_task_instances(results,session,dag=self.dag)session.commit()returncount
@provide_session
[docs]defget_task_instances(self,start_date:Optional[datetime]=None,end_date:Optional[datetime]=None,session:Session=None,)->List[TaskInstance]:""" Get a set of task instance related to this task for a specific date range. """end_date=end_dateortimezone.utcnow()return(session.query(TaskInstance).filter(TaskInstance.dag_id==self.dag_id).filter(TaskInstance.task_id==self.task_id).filter(TaskInstance.execution_date>=start_date).filter(TaskInstance.execution_date<=end_date).order_by(TaskInstance.execution_date)
.all())
[docs]defget_flat_relative_ids(self,upstream:bool=False,found_descendants:Optional[Set[str]]=None,)->Set[str]:"""Get a flat set of relatives' ids, either upstream or downstream."""ifnotself._dag:returnset()ifnotfound_descendants:found_descendants=set()relative_ids=self.get_direct_relative_ids(upstream)forrelative_idinrelative_ids:ifrelative_idnotinfound_descendants:found_descendants.add(relative_id)relative_task=self._dag.task_dict[relative_id]relative_task.get_flat_relative_ids(upstream,found_descendants)returnfound_descendants
[docs]defget_flat_relatives(self,upstream:bool=False):"""Get a flat list of relatives, either upstream or downstream."""ifnotself._dag:returnset()fromairflow.models.dagimportDAGdag:DAG=self._dagreturnlist(map(lambdatask_id:dag.task_dict[task_id],self.get_flat_relative_ids(upstream)))
@provide_session
[docs]defrun(self,start_date:Optional[datetime]=None,end_date:Optional[datetime]=None,ignore_first_depends_on_past:bool=True,ignore_ti_state:bool=False,mark_success:bool=False,test_mode:bool=False,session:Session=None,)->None:"""Run a set of task instances for a date range."""fromairflow.modelsimportDagRunfromairflow.utils.typesimportDagRunTypestart_date=start_dateorself.start_dateend_date=end_dateorself.end_dateortimezone.utcnow()forinfoinself.dag.iter_dagrun_infos_between(start_date,end_date,align=False):ignore_depends_on_past=info.logical_date==start_dateandignore_first_depends_on_pasttry:dag_run=(session.query(DagRun).filter(DagRun.dag_id==self.dag_id,DagRun.execution_date==info.logical_date,).one())ti=TaskInstance(self,run_id=dag_run.run_id)exceptNoResultFound:# This is _mostly_ only used in testsdr=DagRun(dag_id=self.dag_id,run_id=DagRun.generate_run_id(DagRunType.MANUAL,info.logical_date),run_type=DagRunType.MANUAL,execution_date=info.logical_date,data_interval=info.data_interval,)ti=TaskInstance(self,run_id=None)ti.dag_run=drsession.add(dr)session.flush()ti.run(mark_success=mark_success,ignore_depends_on_past=ignore_depends_on_past,ignore_ti_state=ignore_ti_state,test_mode=test_mode,session=session,
)
[docs]defdry_run(self)->None:"""Performs dry run for the operator - just render template fields."""self.log.info('Dry run')forfieldinself.template_fields:content=getattr(self,field)ifcontentandisinstance(content,str):self.log.info('Rendering template for %s',field)self.log.info(content)
[docs]defget_direct_relative_ids(self,upstream:bool=False)->Set[str]:""" Get set of the direct relative ids to the current task, upstream or downstream. """ifupstream:returnself._upstream_task_idselse:returnself._downstream_task_ids
[docs]defget_direct_relatives(self,upstream:bool=False)->List["BaseOperator"]:""" Get list of the direct relatives to the current task, upstream or downstream. """ifupstream:returnself.upstream_listelse:returnself.downstream_list
[docs]deftask_type(self)->str:"""@property: type of the task"""returnself.__class__.__name__
[docs]defadd_only_new(self,item_set:Set[str],item:str,dag_id:str)->None:"""Adds only new items to item set"""ifiteminitem_set:self.log.warning('Dependency %s, %s already registered for DAG: %s',self,item,dag_id)else:item_set.add(item)
@property
[docs]defroots(self)->List["BaseOperator"]:"""Required by TaskMixin"""return[self]
@property
[docs]defleaves(self)->List["BaseOperator"]:"""Required by TaskMixin"""return[self]
def_set_relatives(self,task_or_task_list:Union[TaskMixin,Sequence[TaskMixin]],upstream:bool=False,edge_modifier:Optional[EdgeModifier]=None,)->None:"""Sets relatives for the task or task list."""ifnotisinstance(task_or_task_list,Sequence):task_or_task_list=[task_or_task_list]task_list:List["BaseOperator"]=[]fortask_objectintask_or_task_list:task_object.update_relative(self,notupstream)relatives=task_object.leavesifupstreamelsetask_object.rootstask_list.extend(relatives)fortaskintask_list:ifnotisinstance(task,BaseOperator):raiseAirflowException("Relationships can only be set between ""Operators; received {}".format(task.__class__.__name__))# relationships can only be set if the tasks share a single DAG. Tasks# without a DAG are assigned to that DAG.dags={task._dag.dag_id:task._dagfortaskinself.roots+task_listiftask.has_dag()# type: ignore}iflen(dags)>1:raiseAirflowException(f'Tried to set relationships between tasks in more than one DAG: {dags.values()}')eliflen(dags)==1:dag=dags.popitem()[1]else:raiseAirflowException("Tried to create relationships between tasks that don't have ""DAGs yet. Set the DAG for at least one ""task and try again: {}".format([self]+task_list))ifdagandnotself.has_dag():# If this task does not yet have a dag, add it to the same dag as the other task and# put it in the dag's root TaskGroup.self.dag=dagself.dag.task_group.add(self)fortaskintask_list:ifdagandnottask.has_dag():# If the other task does not yet have a dag, add it to the same dag as this task and# put it in the dag's root TaskGroup.task.dag=dagtask.dag.task_group.add(task)ifupstream:task.add_only_new(task.get_direct_relative_ids(upstream=False),self.task_id,self.dag.dag_id)self.add_only_new(self._upstream_task_ids,task.task_id,task.dag.dag_id)ifedge_modifier:edge_modifier.add_edge_info(self.dag,task.task_id,self.task_id)else:self.add_only_new(self._downstream_task_ids,task.task_id,task.dag.dag_id)task.add_only_new(task.get_direct_relative_ids(upstream=True),self.task_id,self.dag.dag_id)ifedge_modifier:edge_modifier.add_edge_info(self.dag,self.task_id,task.task_id)
[docs]defset_downstream(self,task_or_task_list:Union[TaskMixin,Sequence[TaskMixin]],edge_modifier:Optional[EdgeModifier]=None,)->None:""" Set a task or a task list to be directly downstream from the current task. Required by TaskMixin. """self._set_relatives(task_or_task_list,upstream=False,edge_modifier=edge_modifier)
[docs]defset_upstream(self,task_or_task_list:Union[TaskMixin,Sequence[TaskMixin]],edge_modifier:Optional[EdgeModifier]=None,)->None:""" Set a task or a task list to be directly upstream from the current task. Required by TaskMixin. """self._set_relatives(task_or_task_list,upstream=True,edge_modifier=edge_modifier)
@property
[docs]defoutput(self):"""Returns reference to XCom pushed by current operator"""fromairflow.models.xcom_argimportXComArgreturnXComArg(operator=self)
@staticmethod
[docs]defxcom_push(context:Any,key:str,value:Any,execution_date:Optional[datetime]=None,)->None:""" Make an XCom available for tasks to pull. :param context: Execution Context Dictionary :type: Any :param key: A key for the XCom :type key: str :param value: A value for the XCom. The value is pickled and stored in the database. :type value: any pickleable object :param execution_date: if provided, the XCom will not be visible until this date. This can be used, for example, to send a message to a task on a future date without it being immediately visible. :type execution_date: datetime """context['ti'].xcom_push(key=key,value=value,execution_date=execution_date)
@staticmethod
[docs]defxcom_pull(context:Any,task_ids:Optional[List[str]]=None,dag_id:Optional[str]=None,key:str=XCOM_RETURN_KEY,include_prior_dates:Optional[bool]=None,)->Any:""" Pull XComs that optionally meet certain criteria. The default value for `key` limits the search to XComs that were returned by other tasks (as opposed to those that were pushed manually). To remove this filter, pass key=None (or any desired value). If a single task_id string is provided, the result is the value of the most recent matching XCom from that task_id. If multiple task_ids are provided, a tuple of matching values is returned. None is returned whenever no matches are found. :param context: Execution Context Dictionary :type: Any :param key: A key for the XCom. If provided, only XComs with matching keys will be returned. The default key is 'return_value', also available as a constant XCOM_RETURN_KEY. This key is automatically given to XComs returned by tasks (as opposed to being pushed manually). To remove the filter, pass key=None. :type key: str :param task_ids: Only XComs from tasks with matching ids will be pulled. Can pass None to remove the filter. :type task_ids: str or iterable of strings (representing task_ids) :param dag_id: If provided, only pulls XComs from this DAG. If None (default), the DAG of the calling task is used. :type dag_id: str :param include_prior_dates: If False, only XComs from the current execution_date are returned. If True, XComs from previous dates are returned as well. :type include_prior_dates: bool """returncontext['ti'].xcom_pull(key=key,task_ids=task_ids,dag_id=dag_id,include_prior_dates=include_prior_dates
)@cached_property
[docs]defextra_links(self)->List[str]:"""@property: extra links for the task"""returnlist(set(self.operator_extra_link_dict.keys()).union(self.global_operator_extra_link_dict.keys())
)
[docs]defget_extra_links(self,dttm:datetime,link_name:str)->Optional[Dict[str,Any]]:""" For an operator, gets the URL that the external links specified in `extra_links` should point to. :raise ValueError: The error message of a ValueError will be passed on through to the fronted to show up as a tooltip on the disabled link :param dttm: The datetime parsed execution date for the URL being searched for :param link_name: The name of the link we're looking for the URL for. Should be one of the options specified in `extra_links` :return: A URL """iflink_nameinself.operator_extra_link_dict:returnself.operator_extra_link_dict[link_name].get_link(self,dttm)eliflink_nameinself.global_operator_extra_link_dict:returnself.global_operator_extra_link_dict[link_name].get_link(self,dttm)else:returnNone
@classmethod
[docs]defget_serialized_fields(cls):"""Stringified DAGs and operators contain exactly these fields."""ifnotcls.__serialized_fields:fromairflow.models.dagimportDagContext# make sure the following dummy task is not added to current active# dag in context, otherwise, it will result in# `RuntimeError: dictionary changed size during iteration`# Exception in SerializedDAG.serialize_dag() call.DagContext.push_context_managed_dag(None)cls.__serialized_fields=frozenset(vars(BaseOperator(task_id='test')).keys()-{'inlets','outlets','_upstream_task_ids','default_args','dag','_dag','_BaseOperator__instantiated',}|{'_task_type','subdag','ui_color','ui_fgcolor','template_ext','template_fields','template_fields_renderers','params',})DagContext.pop_context_managed_dag()returncls.__serialized_fields
[docs]defis_smart_sensor_compatible(self):"""Return if this operator can use smart service. Default False."""returnFalse
@property
[docs]definherits_from_dummy_operator(self):"""Used to determine if an Operator is inherited from DummyOperator"""# This looks like `isinstance(self, DummyOperator) would work, but this also# needs to cope when `self` is a Serialized instance of a DummyOperator or one# of its sub-classes (which don't inherit from anything but BaseOperator).returngetattr(self,'_is_dummy',False)
[docs]defdefer(self,*,trigger:BaseTrigger,method_name:str,kwargs:Optional[Dict[str,Any]]=None,timeout:Optional[timedelta]=None,):""" Marks this Operator as being "deferred" - that is, suspending its execution until the provided trigger fires an event. This is achieved by raising a special exception (TaskDeferred) which is caught in the main _execute_task wrapper. """raiseTaskDeferred(trigger=trigger,method_name=method_name,kwargs=kwargs,timeout=timeout)
""" This property will be used by Airflow Plugins to find the Operators to which you want to assign this Operator Link :return: List of Operator classes used by task for which you want to create extra link """@property@abstractmethod