From commits-return-27138-archive-asf-public=cust-asf.ponee.io@airflow.incubator.apache.org Mon Nov 5 19:36:49 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id B6B8C18067A for ; Mon, 5 Nov 2018 19:36:48 +0100 (CET) Received: (qmail 32252 invoked by uid 500); 5 Nov 2018 18:36:47 -0000 Mailing-List: contact commits-help@airflow.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@airflow.incubator.apache.org Delivered-To: mailing list commits@airflow.incubator.apache.org Received: (qmail 32243 invoked by uid 99); 5 Nov 2018 18:36:47 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 05 Nov 2018 18:36:47 +0000 From: GitBox To: commits@airflow.apache.org Subject: [GitHub] Fokko closed pull request #4058: [AIRFLOW-3218] add support for poking a whole DAG Message-ID: <154144300728.22627.13029382985553994707.gitbox@gitbox.apache.org> Date: Mon, 05 Nov 2018 18:36:47 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Fokko closed pull request #4058: [AIRFLOW-3218] add support for poking a whole DAG URL: https://github.com/apache/incubator-airflow/pull/4058 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/airflow/sensors/external_task_sensor.py b/airflow/sensors/external_task_sensor.py index c80986fa9f..b986c19c79 100644 --- a/airflow/sensors/external_task_sensor.py +++ b/airflow/sensors/external_task_sensor.py @@ -17,7 +17,7 @@ # specific language governing permissions and limitations # under the License. -from airflow.models import TaskInstance +from airflow.models import TaskInstance, DagRun from airflow.sensors.base_sensor_operator import BaseSensorOperator from airflow.utils.db import provide_session from airflow.utils.decorators import apply_defaults @@ -26,18 +26,18 @@ class ExternalTaskSensor(BaseSensorOperator): """ - Waits for a task to complete in a different DAG + Waits for a different DAG or a task in in a different DAG to complete :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 + 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. + 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. @@ -89,13 +89,23 @@ def poke(self, context, session=None): '{self.external_dag_id}.' '{self.external_task_id} on ' '{} ... '.format(serialized_dttm_filter, **locals())) - TI = TaskInstance - 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() + if self.external_task_id: + TI = TaskInstance + + 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: + DR = DagRun + 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) diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index feb0cdcd40..b54f811d4d 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -17,7 +17,6 @@ # specific language governing permissions and limitations # under the License. import unittest - from datetime import timedelta, time from airflow import DAG, configuration, settings @@ -75,6 +74,30 @@ def test_external_task_sensor(self): ignore_ti_state=True ) + def test_external_dag_sensor(self): + + other_dag = DAG( + 'other_dag', + default_args=self.args, + end_date=DEFAULT_DATE, + schedule_interval='@once') + other_dag.create_dagrun( + run_id='test', + start_date=DEFAULT_DATE, + execution_date=DEFAULT_DATE, + state=State.SUCCESS) + t = ExternalTaskSensor( + task_id='test_external_dag_sensor_check', + external_dag_id='other_dag', + external_task_id=None, + dag=self.dag + ) + t.run( + start_date=DEFAULT_DATE, + end_date=DEFAULT_DATE, + ignore_ti_state=True + ) + def test_templated_sensor(self): dag = DAG(TEST_DAG_ID, self.args) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org With regards, Apache Git Services