## 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__importannotationsimportdatetimeimportjsonimporttimeimportwarningsfromtypingimportTYPE_CHECKING,Any,Sequence,castfromsqlalchemyimportselectfromsqlalchemy.orm.excimportNoResultFoundfromairflow.api.common.trigger_dagimporttrigger_dagfromairflow.api_internal.internal_api_callimportInternalApiConfigfromairflow.configurationimportconffromairflow.exceptionsimport(AirflowException,AirflowSkipException,DagNotFound,DagRunAlreadyExists,RemovedInAirflow3Warning,)fromairflow.models.baseoperatorimportBaseOperatorfromairflow.models.baseoperatorlinkimportBaseOperatorLinkfromairflow.models.dagimportDagModelfromairflow.models.dagbagimportDagBagfromairflow.models.dagrunimportDagRunfromairflow.models.xcomimportXComfromairflow.triggers.external_taskimportDagStateTriggerfromairflow.utilsimporttimezonefromairflow.utils.helpersimportbuild_airflow_url_with_queryfromairflow.utils.sessionimportprovide_sessionfromairflow.utils.stateimportDagRunStatefromairflow.utils.typesimportDagRunType
[docs]classTriggerDagRunLink(BaseOperatorLink):""" Operator link for TriggerDagRunOperator. It allows users to access DAG triggered by task using TriggerDagRunOperator. """
[docs]defget_link(self,operator:BaseOperator,*,ti_key:TaskInstanceKey)->str:# Fetch the correct dag_run_id for the triggerED dag which is# stored in xcom during execution of the triggerING task.triggered_dag_run_id=XCom.get_value(ti_key=ti_key,key=XCOM_RUN_ID)query={"dag_id":cast(TriggerDagRunOperator,operator).trigger_dag_id,"dag_run_id":triggered_dag_run_id,}returnbuild_airflow_url_with_query(query)
[docs]classTriggerDagRunOperator(BaseOperator):""" Triggers a DAG run for a specified DAG ID. Note that if database isolation mode is enabled, not all features are supported. :param trigger_dag_id: The ``dag_id`` of the DAG to trigger (templated). :param trigger_run_id: The run ID to use for the triggered DAG run (templated). If not provided, a run ID will be automatically generated. :param conf: Configuration for the DAG run (templated). :param logical_date: Logical date for the triggered DAG (templated). :param reset_dag_run: Whether clear existing DAG run if already exists. This is useful when backfill or rerun an existing DAG run. This only resets (not recreates) the DAG run. DAG run conf is immutable and will not be reset on rerun of an existing DAG run. When reset_dag_run=False and dag run exists, DagRunAlreadyExists will be raised. When reset_dag_run=True and dag run exists, existing DAG run will be cleared to rerun. :param wait_for_completion: Whether or not wait for DAG run completion. (default: False) :param poke_interval: Poke interval to check DAG run status when wait_for_completion=True. (default: 60) :param allowed_states: Optional list of allowed DAG run states of the triggered DAG. This is useful when setting ``wait_for_completion`` to True. Must be a valid DagRunState. Default is ``[DagRunState.SUCCESS]``. :param failed_states: Optional list of failed or disallowed DAG run states of the triggered DAG. This is useful when setting ``wait_for_completion`` to True. Must be a valid DagRunState. Default is ``[DagRunState.FAILED]``. :param skip_when_already_exists: Set to true to mark the task as SKIPPED if a DAG run of the triggered DAG for the same logical date already exists. :param deferrable: If waiting for completion, whether or not to defer the task until done, default is ``False``. :param execution_date: Deprecated parameter; same as ``logical_date``. """
def__init__(self,*,trigger_dag_id:str,trigger_run_id:str|None=None,conf:dict|None=None,logical_date:str|datetime.datetime|None=None,reset_dag_run:bool=False,wait_for_completion:bool=False,poke_interval:int=60,allowed_states:list[str|DagRunState]|None=None,failed_states:list[str|DagRunState]|None=None,skip_when_already_exists:bool=False,deferrable:bool=conf.getboolean("operators","default_deferrable",fallback=False),execution_date:str|datetime.datetime|None=None,**kwargs,)->None:super().__init__(**kwargs)self.trigger_dag_id=trigger_dag_idself.trigger_run_id=trigger_run_idself.conf=confself.reset_dag_run=reset_dag_runself.wait_for_completion=wait_for_completionself.poke_interval=poke_intervalifallowed_states:self.allowed_states=[DagRunState(s)forsinallowed_states]else:self.allowed_states=[DagRunState.SUCCESS]iffailed_statesorfailed_states==[]:self.failed_states=[DagRunState(s)forsinfailed_states]else:self.failed_states=[DagRunState.FAILED]self.skip_when_already_exists=skip_when_already_existsself._defer=deferrableifexecution_dateisnotNone:warnings.warn("Parameter 'execution_date' is deprecated. Use 'logical_date' instead.",RemovedInAirflow3Warning,stacklevel=2,)logical_date=execution_dateiflogical_dateisnotNoneandnotisinstance(logical_date,(str,datetime.datetime)):type_name=type(logical_date).__name__raiseTypeError(f"Expected str or datetime.datetime type for parameter 'logical_date'. Got {type_name}")self.logical_date=logical_date
[docs]defexecute(self,context:Context):ifInternalApiConfig.get_use_internal_api():ifself.reset_dag_run:raiseAirflowException("Parameter reset_dag_run=True is broken with Database Isolation Mode.")ifself.wait_for_completion:raiseAirflowException("Parameter wait_for_completion=True is broken with Database Isolation Mode.")ifisinstance(self.logical_date,datetime.datetime):parsed_logical_date=self.logical_dateelifisinstance(self.logical_date,str):parsed_logical_date=timezone.parse(self.logical_date)else:parsed_logical_date=timezone.utcnow()try:json.dumps(self.conf)exceptTypeError:raiseAirflowException("conf parameter should be JSON Serializable")ifself.trigger_run_id:run_id=str(self.trigger_run_id)else:run_id=DagRun.generate_run_id(DagRunType.MANUAL,parsed_logical_date)try:dag_run=trigger_dag(dag_id=self.trigger_dag_id,run_id=run_id,conf=self.conf,execution_date=parsed_logical_date,replace_microseconds=False,)exceptDagRunAlreadyExistsase:ifself.reset_dag_run:dag_run=e.dag_runself.log.info("Clearing %s on %s",self.trigger_dag_id,dag_run.logical_date)# Get target dag object and call clear()dag_model=DagModel.get_current(self.trigger_dag_id)ifdag_modelisNone:raiseDagNotFound(f"Dag id {self.trigger_dag_id} not found in DagModel")# Note: here execution fails on database isolation mode. Needs structural changes for AIP-72dag_bag=DagBag(dag_folder=dag_model.fileloc,read_dags_from_db=True)dag=dag_bag.get_dag(self.trigger_dag_id)dag.clear(start_date=dag_run.logical_date,end_date=dag_run.logical_date)else:ifself.skip_when_already_exists:raiseAirflowSkipException("Skipping due to skip_when_already_exists is set to True and DagRunAlreadyExists")raiseeifdag_runisNone:raiseRuntimeError("The dag_run should be set here!")# Store the run id from the dag run (either created or found above) to# be used when creating the extra link on the webserver.# TODO: Logical date as xcom stored only for backwards compatibility. Remove in Airflow 3.0ti=context["task_instance"]ti.xcom_push(key=XCOM_LOGICAL_DATE_ISO,value=dag_run.logical_date.isoformat())ti.xcom_push(key=XCOM_RUN_ID,value=dag_run.run_id)ifself.wait_for_completion:# Kick off the deferral processifself._defer:self.defer(trigger=DagStateTrigger(dag_id=self.trigger_dag_id,states=self.allowed_states+self.failed_states,execution_dates=[dag_run.logical_date],poll_interval=self.poke_interval,),method_name="execute_complete",)# wait for dag to completewhileTrue:self.log.info("Waiting for %s on %s to become allowed state %s ...",self.trigger_dag_id,dag_run.logical_date,self.allowed_states,)time.sleep(self.poke_interval)# Note: here execution fails on database isolation mode. Needs structural changes for AIP-72dag_run.refresh_from_db()state=dag_run.stateifstateinself.failed_states:raiseAirflowException(f"{self.trigger_dag_id} failed with failed states {state}")ifstateinself.allowed_states:self.log.info("%s finished with allowed state %s",self.trigger_dag_id,state)return
@provide_session
[docs]defexecute_complete(self,context:Context,session:Session,event:tuple[str,dict[str,Any]]):# This logical_date is parsed from the return trigger eventprovided_logical_date=event[1]["execution_dates"][0]try:# Note: here execution fails on database isolation mode. Needs structural changes for AIP-72dag_run=session.execute(select(DagRun).where(DagRun.dag_id==self.trigger_dag_id,DagRun.execution_date==provided_logical_date)).scalar_one()exceptNoResultFound:raiseAirflowException(f"No DAG run found for DAG {self.trigger_dag_id} and logical date {self.logical_date}")state=dag_run.stateifstateinself.failed_states:raiseAirflowException(f"{self.trigger_dag_id} failed with failed state {state}")ifstateinself.allowed_states:self.log.info("%s finished with allowed state %s",self.trigger_dag_id,state)returnraiseAirflowException(f"{self.trigger_dag_id} return {state} which is not in {self.failed_states}"f" or {self.allowed_states}")