Source code for airflow.sensors.external_task_sensor

# -*- coding: utf-8 -*-
#
# 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.

import os

from airflow.exceptions import AirflowException
from airflow.models import TaskInstance, DagBag, DagModel, DagRun
from airflow.sensors.base_sensor_operator import BaseSensorOperator
from airflow.utils.db import provide_session
from airflow.utils.decorators import apply_defaults
from airflow.utils.state import State


[docs]class ExternalTaskSensor(BaseSensorOperator): """ Waits for a different DAG or a task in a different DAG to complete for a specific execution_date :param external_dag_id: The dag_id that contains the task you want to wait for :type external_dag_id: str :param external_task_id: The task_id that contains the task you want to wait for. If ``None`` the sensor waits for the DAG :type external_task_id: str :param allowed_states: list of allowed states, default is ``['success']`` :type allowed_states: list :param execution_delta: time difference with the previous execution to look at, the default is the same execution_date as the current task or DAG. For yesterday, use [positive!] datetime.timedelta(days=1). Either execution_delta or execution_date_fn can be passed to ExternalTaskSensor, but not both. :type execution_delta: datetime.timedelta :param execution_date_fn: function that receives the current execution date and returns the desired execution dates to query. Either execution_delta or execution_date_fn can be passed to ExternalTaskSensor, but not both. :type execution_date_fn: callable :param check_existence: Set to `True` to check if the external task exists (when external_task_id is not None) or check if the DAG to wait for exists (when external_task_id is None), and immediately cease waiting if the external task or DAG does not exist (default value: False). :type check_existence: bool """
[docs] template_fields = ['external_dag_id', 'external_task_id']
[docs] ui_color = '#19647e'
@apply_defaults def __init__(self, external_dag_id, external_task_id, allowed_states=None, execution_delta=None, execution_date_fn=None, check_existence=False, *args, **kwargs): super(ExternalTaskSensor, self).__init__(*args, **kwargs) self.allowed_states = allowed_states or [State.SUCCESS] if external_task_id: if not set(self.allowed_states) <= set(State.task_states): raise ValueError( 'Valid values for `allowed_states` ' 'when `external_task_id` is not `None`: {}'.format(State.task_states) ) else: if not set(self.allowed_states) <= set(State.dag_states): raise ValueError( 'Valid values for `allowed_states` ' 'when `external_task_id` is `None`: {}'.format(State.dag_states) ) if execution_delta is not None and execution_date_fn is not None: raise ValueError( 'Only one of `execution_delta` or `execution_date_fn` may ' 'be provided to ExternalTaskSensor; not both.') self.execution_delta = execution_delta self.execution_date_fn = execution_date_fn self.external_dag_id = external_dag_id self.external_task_id = external_task_id self.check_existence = check_existence @provide_session
[docs] def poke(self, context, session=None): if self.execution_delta: dttm = context['execution_date'] - self.execution_delta elif self.execution_date_fn: dttm = self.execution_date_fn(context['execution_date']) else: dttm = context['execution_date'] dttm_filter = dttm if isinstance(dttm, list) else [dttm] serialized_dttm_filter = ','.join( [datetime.isoformat() for datetime in dttm_filter]) self.log.info( 'Poking for %s.%s on %s ... ', self.external_dag_id, self.external_task_id, serialized_dttm_filter ) DM = DagModel TI = TaskInstance DR = DagRun if self.check_existence: dag_to_wait = session.query(DM).filter( DM.dag_id == self.external_dag_id ).first() if not dag_to_wait: raise AirflowException('The external DAG ' '{} does not exist.'.format(self.external_dag_id)) else: if not os.path.exists(dag_to_wait.fileloc): raise AirflowException('The external DAG ' '{} was deleted.'.format(self.external_dag_id)) if self.external_task_id: refreshed_dag_info = DagBag(dag_to_wait.fileloc).get_dag(self.external_dag_id) if not refreshed_dag_info.has_task(self.external_task_id): raise AirflowException('The external task' '{} in DAG {} does not exist.'.format(self.external_task_id, self.external_dag_id)) if self.external_task_id: count = session.query(TI).filter( TI.dag_id == self.external_dag_id, TI.task_id == self.external_task_id, TI.state.in_(self.allowed_states), TI.execution_date.in_(dttm_filter), ).count() else: count = session.query(DR).filter( DR.dag_id == self.external_dag_id, DR.state.in_(self.allowed_states), DR.execution_date.in_(dttm_filter), ).count() session.commit() return count == len(dttm_filter)

Was this entry helpful?