## 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__importannotationsimportdatetimeimportinspectfromtypingimportTYPE_CHECKING,Any,Callable,ClassVar,Collection,Iterable,Iterator,Sequencefromairflow.compat.functoolsimportcache,cached_propertyfromairflow.configurationimportconffromairflow.exceptionsimportAirflowExceptionfromairflow.models.expandinputimportNotFullyPopulatedfromairflow.models.taskmixinimportDAGNodefromairflow.utils.contextimportContextfromairflow.utils.helpersimportrender_template_as_native,render_template_to_stringfromairflow.utils.log.logging_mixinimportLoggingMixinfromairflow.utils.log.secrets_maskerimportredactfromairflow.utils.mixinsimportResolveMixinfromairflow.utils.sessionimportNEW_SESSION,provide_sessionfromairflow.utils.sqlalchemyimportskip_locked,with_row_locksfromairflow.utils.stateimportState,TaskInstanceStatefromairflow.utils.task_groupimportMappedTaskGroupfromairflow.utils.trigger_ruleimportTriggerRulefromairflow.utils.weight_ruleimportWeightRule
[docs]classNotMapped(Exception):"""Raise if a task is neither mapped nor has any parent mapped groups."""
[docs]classAbstractOperator(LoggingMixin,DAGNode):"""Common implementation for operators, including unmapped and mapped. This base class is more about sharing implementations, not defining a common interface. Unfortunately it's difficult to use this as the common base class for typing due to BaseOperator carrying too much historical baggage. The union type ``from airflow.models.operator import Operator`` is easier to use for typing purposes. :meta private: """
[docs]HIDE_ATTRS_FROM_UI:ClassVar[frozenset[str]]=frozenset(("log","dag",# We show dag_id, don't need to show this too"node_id",# Duplicates task_id"task_group",# Doesn't have a useful repr, no point showing in UI"inherits_from_empty_operator",# impl detail# For compatibility with TG, for operators these are just the current task, no point showing"roots","leaves",# These lists are already shown via *_task_ids"upstream_list","downstream_list",# Not useful, implementation detail, already shown elsewhere"global_operator_extra_link_dict","operator_extra_link_dict",
[docs]defget_template_env(self)->jinja2.Environment:"""Fetch a Jinja template environment from the DAG or instantiate empty environment if no DAG."""# This is imported locally since Jinja2 is heavy and we don't need it# for most of the functionalities. It is imported by get_template_env()# though, so we don't need to put this after the 'if dag' check.fromairflow.templatesimportSandboxedEnvironmentdag=self.get_dag()ifdag:returndag.get_template_env(force_sandboxed=False)returnSandboxedEnvironment(cache_size=0)
[docs]defprepare_template(self)->None:"""Hook 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])# type: ignoreexceptException:self.log.exception("Failed to resolve template field %r",field)elifisinstance(content,list):env=self.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]# type: ignoreexceptException:self.log.exception("Failed to get source %s",item)self.prepare_template()
[docs]defget_direct_relative_ids(self,upstream:bool=False)->set[str]:"""Get direct relative IDs to the current task, upstream or downstream."""ifupstream:returnself.upstream_task_idsreturnself.downstream_task_ids
[docs]defget_flat_relative_ids(self,upstream:bool=False,found_descendants:set[str]|None=None,)->set[str]:"""Get a flat set of relative IDs, upstream or downstream."""dag=self.get_dag()ifnotdag:returnset()iffound_descendantsisNone:found_descendants=set()task_ids_to_trace=self.get_direct_relative_ids(upstream)whiletask_ids_to_trace:task_ids_to_trace_next:set[str]=set()fortask_idintask_ids_to_trace:iftask_idinfound_descendants:continuetask_ids_to_trace_next.update(dag.task_dict[task_id].get_direct_relative_ids(upstream))found_descendants.add(task_id)task_ids_to_trace=task_ids_to_trace_nextreturnfound_descendants
[docs]defget_flat_relatives(self,upstream:bool=False)->Collection[Operator]:"""Get a flat list of relatives, either upstream or downstream."""dag=self.get_dag()ifnotdag:returnset()return[dag.task_dict[task_id]fortask_idinself.get_flat_relative_ids(upstream)]
def_iter_all_mapped_downstreams(self)->Iterator[MappedOperator|MappedTaskGroup]:"""Return mapped nodes that are direct dependencies of the current task. For now, this walks the entire DAG to find mapped nodes that has this current task as an upstream. We cannot use ``downstream_list`` since it only contains operators, not task groups. In the future, we should provide a way to record an DAG node's all downstream nodes instead. Note that this does not guarantee the returned tasks actually use the current task for task mapping, but only checks those task are mapped operators, and are downstreams of the current task. To get a list of tasks that uses the current task for task mapping, use :meth:`iter_mapped_dependants` instead. """fromairflow.models.mappedoperatorimportMappedOperatorfromairflow.utils.task_groupimportTaskGroupdef_walk_group(group:TaskGroup)->Iterable[tuple[str,DAGNode]]:"""Recursively walk children in a task group. This yields all direct children (including both tasks and task groups), and all children of any task groups. """forkey,childingroup.children.items():yieldkey,childifisinstance(child,TaskGroup):yield from_walk_group(child)dag=self.get_dag()ifnotdag:raiseRuntimeError("Cannot check for mapped dependants when not attached to a DAG")forkey,childin_walk_group(dag.task_group):ifkey==self.node_id:continueifnotisinstance(child,(MappedOperator,MappedTaskGroup)):continueifself.node_idinchild.upstream_task_ids:yieldchild
[docs]defiter_mapped_dependants(self)->Iterator[MappedOperator|MappedTaskGroup]:"""Return mapped nodes that depend on the current task the expansion. For now, this walks the entire DAG to find mapped nodes that has this current task as an upstream. We cannot use ``downstream_list`` since it only contains operators, not task groups. In the future, we should provide a way to record an DAG node's all downstream nodes instead. """return(downstreamfordownstreaminself._iter_all_mapped_downstreams()ifany(p.node_id==self.node_idforpindownstream.iter_mapped_dependencies())
)
[docs]defiter_mapped_task_groups(self)->Iterator[MappedTaskGroup]:"""Return mapped task groups this task belongs to. Groups are returned from the closest to the outmost. :meta private: """parent=self.task_groupwhileparentisnotNone:ifisinstance(parent,MappedTaskGroup):yieldparentparent=parent.task_group
[docs]defunmap(self,resolve:None|dict[str,Any]|tuple[Context,Session])->BaseOperator:"""Get the "normal" operator from current abstract operator. MappedOperator uses this to unmap itself based on the map index. A non- mapped operator (i.e. BaseOperator subclass) simply returns itself. :meta private: """raiseNotImplementedError()
@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=Falsedag=self.get_dag()ifdagisNone:returnself.priority_weightreturnself.priority_weight+sum(dag.task_dict[task_id].priority_weightfortask_idinself.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.operator_classinope.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}
[docs]defget_extra_links(self,ti:TaskInstance,link_name:str)->str|None:"""For an operator, gets the URLs that the ``extra_links`` entry points to. :meta private: :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 ti: The TaskInstance 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``. """link:BaseOperatorLink|None=self.operator_extra_link_dict.get(link_name)ifnotlink:link=self.global_operator_extra_link_dict.get(link_name)ifnotlink:returnNoneparameters=inspect.signature(link.get_link).parametersold_signature=all(name!="ti_key"forname,pinparameters.items()ifp.kind!=p.VAR_KEYWORD)ifold_signature:returnlink.get_link(self.unmap(None),ti.dag_run.logical_date)# type: ignore[misc]returnlink.get_link(self.unmap(None),ti_key=ti.key)
@cache
[docs]defget_parse_time_mapped_ti_count(self)->int:"""Number of mapped task instances that can be created on DAG run creation. This only considers literal mapped arguments, and would return *None* when any non-literal values are used for mapping. :raise NotFullyPopulated: If non-literal mapped arguments are encountered. :raise NotMapped: If the operator is neither mapped, nor has any parent mapped task groups. :return: Total number of mapped TIs this task should have. """group=self.get_closest_mapped_task_group()ifgroupisNone:raiseNotMappedreturngroup.get_parse_time_mapped_ti_count()
[docs]defget_mapped_ti_count(self,run_id:str,*,session:Session)->int:"""Number of mapped TaskInstances that can be created at run time. This considers both literal and non-literal mapped arguments, and the result is therefore available when all depended tasks have finished. The return value should be identical to ``parse_time_mapped_ti_count`` if all mapped arguments are literal. :raise NotFullyPopulated: If upstream tasks are not all complete yet. :raise NotMapped: If the operator is neither mapped, nor has any parent mapped task groups. :return: Total number of mapped TIs this task should have. """group=self.get_closest_mapped_task_group()ifgroupisNone:raiseNotMappedreturngroup.get_mapped_ti_count(run_id,session=session)
[docs]defexpand_mapped_task(self,run_id:str,*,session:Session)->tuple[Sequence[TaskInstance],int]:"""Create the mapped task instances for mapped task. :raise NotMapped: If this task does not need expansion. :return: The newly created mapped task instances (if any) in ascending order by map index, and the maximum map index value. """fromsqlalchemyimportfunc,or_fromairflow.models.baseoperatorimportBaseOperatorfromairflow.models.mappedoperatorimportMappedOperatorfromairflow.models.taskinstanceimportTaskInstancefromairflow.settingsimporttask_instance_mutation_hookifnotisinstance(self,(BaseOperator,MappedOperator)):raiseRuntimeError(f"cannot expand unrecognized operator type {type(self).__name__}")try:total_length:int|None=self.get_mapped_ti_count(run_id,session=session)exceptNotFullyPopulatedase:# It's possible that the upstream tasks are not yet done, but we# don't have upstream of upstreams in partial DAGs (possible in the# mini-scheduler), so we ignore this exception.ifnotself.dagornotself.dag.partial:self.log.error("Cannot expand %r for run %s; missing upstream values: %s",self,run_id,sorted(e.missing),)total_length=Nonestate:TaskInstanceState|None=Noneunmapped_ti:TaskInstance|None=(session.query(TaskInstance).filter(TaskInstance.dag_id==self.dag_id,TaskInstance.task_id==self.task_id,TaskInstance.run_id==run_id,TaskInstance.map_index==-1,or_(TaskInstance.state.in_(State.unfinished),TaskInstance.state.is_(None)),).one_or_none())all_expanded_tis:list[TaskInstance]=[]ifunmapped_ti:# The unmapped task instance still exists and is unfinished, i.e. we# haven't tried to run it before.iftotal_lengthisNone:# If the DAG is partial, it's likely that the upstream tasks# are not done yet, so the task can't fail yet.ifnotself.dagornotself.dag.partial:unmapped_ti.state=TaskInstanceState.UPSTREAM_FAILEDeliftotal_length<1:# If the upstream maps this to a zero-length value, simply mark# the unmapped task instance as SKIPPED (if needed).self.log.info("Marking %s as SKIPPED since the map has %d values to expand",unmapped_ti,total_length,)unmapped_ti.state=TaskInstanceState.SKIPPEDelse:zero_index_ti_exists=(session.query(TaskInstance).filter(TaskInstance.dag_id==self.dag_id,TaskInstance.task_id==self.task_id,TaskInstance.run_id==run_id,TaskInstance.map_index==0,).count()>0)ifnotzero_index_ti_exists:# Otherwise convert this into the first mapped index, and create# TaskInstance for other indexes.unmapped_ti.map_index=0self.log.debug("Updated in place to become %s",unmapped_ti)all_expanded_tis.append(unmapped_ti)session.flush()else:self.log.debug("Deleting the original task instance: %s",unmapped_ti)session.delete(unmapped_ti)state=unmapped_ti.stateiftotal_lengthisNoneortotal_length<1:# Nothing to fixup.indexes_to_map:Iterable[int]=()else:# Only create "missing" ones.current_max_mapping=(session.query(func.max(TaskInstance.map_index)).filter(TaskInstance.dag_id==self.dag_id,TaskInstance.task_id==self.task_id,TaskInstance.run_id==run_id,).scalar())indexes_to_map=range(current_max_mapping+1,total_length)forindexinindexes_to_map:# TODO: Make more efficient with bulk_insert_mappings/bulk_save_mappings.ti=TaskInstance(self,run_id=run_id,map_index=index,state=state)self.log.debug("Expanding TIs upserted %s",ti)task_instance_mutation_hook(ti)ti=session.merge(ti)ti.refresh_from_task(self)# session.merge() loses task information.all_expanded_tis.append(ti)# Coerce the None case to 0 -- these two are almost treated identically,# except the unmapped ti (if exists) is marked to different states.total_expanded_ti_count=total_lengthor0# Any (old) task instances with inapplicable indexes (>= the total# number we need) are set to "REMOVED".query=session.query(TaskInstance).filter(TaskInstance.dag_id==self.dag_id,TaskInstance.task_id==self.task_id,TaskInstance.run_id==run_id,TaskInstance.map_index>=total_expanded_ti_count,)to_update=with_row_locks(query,of=TaskInstance,session=session,**skip_locked(session=session))fortiinto_update:ti.state=TaskInstanceState.REMOVEDsession.flush()returnall_expanded_tis,total_expanded_ti_count-1
[docs]defrender_template_fields(self,context:Context,jinja_env:jinja2.Environment|None=None,)->None:"""Template all attributes listed in *self.template_fields*. If the operator is mapped, this should return the unmapped, fully rendered, and map-expanded operator. The mapped operator should not be modified. However, *context* may be modified in-place to reference the unmapped operator for template rendering. If the operator is not mapped, this should modify the operator in-place. """raiseNotImplementedError()
@provide_sessiondef_do_render_template_fields(self,parent:Any,template_fields:Iterable[str],context:Context,jinja_env:jinja2.Environment,seen_oids:set[int],*,session:Session=NEW_SESSION,)->None:forattr_nameintemplate_fields:try:value=getattr(parent,attr_name)exceptAttributeError:raiseAttributeError(f"{attr_name!r} is configured as a template field "f"but {parent.task_type} does not have this attribute.")ifnotvalue:continuetry:rendered_content=self.render_template(value,context,jinja_env,seen_oids,)exceptException:value_masked=redact(name=attr_name,value=value)self.log.exception("Exception rendering Jinja template for task '%s', field '%s'. Template: %r",self.task_id,attr_name,value_masked,)raiseelse:setattr(parent,attr_name,rendered_content)
[docs]defrender_template(self,content:Any,context:Context,jinja_env:jinja2.Environment|None=None,seen_oids:set[int]|None=None,)->Any:"""Render a templated string. If *content* is a collection holding multiple templated strings, strings in the collection will be templated recursively. :param content: Content to template. Only strings can be templated (may be inside a collection). :param context: Dict with values to apply on templated content :param jinja_env: Jinja environment. Can be provided to avoid re-creating Jinja environments during recursion. :param seen_oids: template fields already rendered (to avoid *RecursionError* on circular dependencies) :return: Templated content """# "content" is a bad name, but we're stuck to it being public API.value=contentdelcontentifseen_oidsisnotNone:oids=seen_oidselse:oids=set()ifid(value)inoids:returnvalueifnotjinja_env:jinja_env=self.get_template_env()ifisinstance(value,str):ifany(value.endswith(ext)forextinself.template_ext):# A filepath.template=jinja_env.get_template(value)else:template=jinja_env.from_string(value)dag=self.get_dag()ifdaganddag.render_template_as_native_obj:returnrender_template_as_native(template,context)returnrender_template_to_string(template,context)ifisinstance(value,ResolveMixin):returnvalue.resolve(context)# Fast path for common built-in collections.ifvalue.__class__istuple:returntuple(self.render_template(element,context,jinja_env,oids)forelementinvalue)elifisinstance(value,tuple):# Special case for named tuples.returnvalue.__class__(*(self.render_template(el,context,jinja_env,oids)forelinvalue))elifisinstance(value,list):return[self.render_template(element,context,jinja_env,oids)forelementinvalue]elifisinstance(value,dict):return{k:self.render_template(v,context,jinja_env,oids)fork,vinvalue.items()}elifisinstance(value,set):return{self.render_template(element,context,jinja_env,oids)forelementinvalue}# More complex collections.self._render_nested_template_fields(value,context,jinja_env,oids)returnvalue
def_render_nested_template_fields(self,value:Any,context:Context,jinja_env:jinja2.Environment,seen_oids:set[int],)->None:ifid(value)inseen_oids:returnseen_oids.add(id(value))try:nested_template_fields=value.template_fieldsexceptAttributeError:# content has no inner template fieldsreturnself._do_render_template_fields(value,nested_template_fields,context,jinja_env,seen_oids)