You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/01/14 19:48:53 UTC

[GitHub] [airflow] hedrickw opened a new issue #13681: ExternalTaskSensor can never find External Parent Task

hedrickw opened a new issue #13681:
URL: https://github.com/apache/airflow/issues/13681


   <!--
   
   Welcome to Apache Airflow!  For a smooth issue process, try to answer the following questions.
   Don't worry if they're not all applicable; just try to include what you can :-)
   
   If you need to include code snippets or logs, please put them in fenced code
   blocks.  If they're super-long, please use the details tag like
   <details><summary>super-long log</summary> lots of stuff </details>
   
   Please delete these comment blocks before submitting the issue.
   
   -->
   
   <!--
   
   IMPORTANT!!!
   
   PLEASE CHECK "SIMILAR TO X EXISTING ISSUES" OPTION IF VISIBLE
   NEXT TO "SUBMIT NEW ISSUE" BUTTON!!!
   
   PLEASE CHECK IF THIS ISSUE HAS BEEN REPORTED PREVIOUSLY USING SEARCH!!!
   
   Please complete the next sections or the issue will be closed.
   These questions are the first thing we need to know to understand the context.
   
   -->
   
   **Apache Airflow version**: 2.0
   
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`): 1.18
   
   **Environment**: Linux 
   
   - **Cloud provider or hardware configuration**: AWS
   - **OS** (e.g. from /etc/os-release): Linux
   - **Kernel** (e.g. `uname -a`):
   - **Install tools**:
   - **Others**:
   
   **What happened**: Following the example below https://github.com/apache/airflow/blob/master/airflow/example_dags/example_external_task_marker_dag.py, when you trigger the parent task, it succeeds so then you trigger the child task it goes into reschedule loop until it times out because the execution date filter is only single list object of the child execution task which wont ever be the same as the parent. https://github.com/apache/airflow/blob/master/airflow/sensors/external_task.py#L216
   
   ```
   IRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=example_external_task_marker_child
   AIRFLOW_CTX_TASK_ID=child_task1
   AIRFLOW_CTX_EXECUTION_DATE=2021-01-14T17:33:14.178268+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-01-14T17:33:14.178268+00:00
   [2021-01-14 17:33:49,750] {external_task.py:153} INFO - Poking for example_external_task_marker_parent.parent_task on 2021-01-14T17:33:14.178268+00:00 ... 
   [2021-01-14 17:33:49,889] {taskinstance.py:1386} INFO - Rescheduling task, marking task as UP_FOR_RESCHEDULE
   [2021-01-14 17:33:49,928] {local_task_job.py:118} INFO - Task exited with return code 0
   [2021-01-14 17:35:22,581] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: example_external_task_marker_child.child_task1 2021-01-14T17:33:14.178268+00:00 [queued]>
   [2021-01-14 17:35:22,626] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: example_external_task_marker_child.child_task1 2021-01-14T17:33:14.178268+00:00 [queued]>
   [2021-01-14 17:35:22,627] {taskinstance.py:1017} INFO - 
   --------------------------------------------------------------------------------
   [2021-01-14 17:35:22,627] {taskinstance.py:1018} INFO - Starting attempt 1 of 1
   [2021-01-14 17:35:22,627] {taskinstance.py:1019} INFO - 
   --------------------------------------------------------------------------------
   [2021-01-14 17:35:22,657] {taskinstance.py:1038} INFO - Executing <Task(ExternalTaskSensor): child_task1> on 2021-01-14T17:33:14.178268+00:00
   [2021-01-14 17:35:22,665] {standard_task_runner.py:51} INFO - Started process 24 to run task
   [2021-01-14 17:35:22,678] {standard_task_runner.py:75} INFO - Running: ['airflow', 'tasks', 'run', 'example_external_task_marker_child', 'child_task1', '2021-01-14T17:33:14.178268+00:00', '--job-id', '53', '--pool', 'default_pool', '--raw', '--subdir', 'DAGS_FOLDER/dags/examples/external_sensor.py', '--cfg-path', '/tmp/tmpzb2z00at']
   [2021-01-14 17:35:22,680] {standard_task_runner.py:76} INFO - Job 53: Subtask child_task1
   [2021-01-14 17:35:22,881] {logging_mixin.py:103} INFO - Running <TaskInstance: example_external_task_marker_child.child_task1 2021-01-14T17:33:14.178268+00:00 [running]> on host exampleexternaltaskmarkerchildchildtask1-e6a0f9157da74583bb7373
   [2021-01-14 17:35:23,667] {taskinstance.py:1230} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=example_external_task_marker_child
   AIRFLOW_CTX_TASK_ID=child_task1
   AIRFLOW_CTX_EXECUTION_DATE=2021-01-14T17:33:14.178268+00:00
   AIRFLOW_CTX_DAG_RUN_ID=manual__2021-01-14T17:33:14.178268+00:00
   [2021-01-14 17:35:23,695] {external_task.py:153} INFO - Poking for example_external_task_marker_parent.parent_task on 2021-01-14T17:33:14.178268+00:00 ... 
   [2021-01-14 17:35:23,833] {taskinstance.py:1386} INFO - Rescheduling task, marking task as UP_FOR_RESCHEDULE
   [2021-01-14 17:35:23,852] {local_task_job.py:118} INFO - Task exited with return code 0
   [2021-01-14 17:36:59,125] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: example_external_task_marker_child.child_task1 2021-01-14T17:33:14.178268+00:00 [queued]>
   [2021-01-14 17:36:59,172] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: example_external_task_marker_child.child_task1 2021-01-14T17:33:14.178268+00:00 [queued]>
   [2021-01-14 17:36:59,172] {taskinstance.py:1017} INFO - 
   ```
   
   <!-- (please include exact error messages if you can) -->
   
   **What you expected to happen**: I would expect the filter to be a range not a single timestamp? Or we should be able to send in a date instead of a datetime?
   
   <!-- What do you think went wrong? -->
   
   **How to reproduce it**:  Run this Example https://github.com/apache/airflow/blob/master/airflow/example_dags/example_external_task_marker_dag.py
   <!---
   
   As minimally and precisely as possible. Keep in mind we do not have access to your cluster or dags.
   
   If you are using kubernetes, please attempt to recreate the issue using minikube or kind.
   
   ## Install minikube/kind
   
   - Minikube https://minikube.sigs.k8s.io/docs/start/
   - Kind https://kind.sigs.k8s.io/docs/user/quick-start/
   
   If this is a UI bug, please provide a screenshot of the bug or a link to a youtube video of the bug in action
   
   You can include images using the .md style of
   ![alt text](http://url/to/img.png)
   
   To record a screencast, mac users can use QuickTime and then create an unlisted youtube video with the resulting .mov file.
   
   --->
   
   
   **Anything else we need to know**:
   
   <!--
   
   How often does this problem occur? Once? Every time etc?
   
   Any relevant logs to include? Put them here in side a detail tag:
   <details><summary>x.log</summary> lots of stuff </details>
   
   -->
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] ephraimbuddy commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760538590


   > @ephraimbuddy Ah maybe its just an outdated file i grabbed example from master and it has schedule of None
   > https://github.com/apache/airflow/blob/master/airflow/example_dags/example_external_task_marker_dag.py
   
   Can you make a PR to correct it? That would be nice


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760531896


   The schedule_interval need to be the same for both DAGs for it to work and shouldn't be None.
   ```python
   #
   # 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.
   
   """
   Example DAG demonstrating setting up inter-DAG dependencies using ExternalTaskSensor and
   ExternalTaskMarker
   
   In this example, child_task1 in example_external_task_marker_child depends on parent_task in
   example_external_task_marker_parent. When parent_task is cleared with "Recursive" selected,
   the presence of ExternalTaskMarker tells Airflow to clear child_task1 and its
   downstream tasks.
   
   ExternalTaskSensor will keep poking for the status of remote ExternalTaskMarker task at a regular
   interval till one of the following will happen:
   1. ExternalTaskMarker reaches the states mentioned in the allowed_states list
       In this case, ExternalTaskSensor will exit with a succes status code
   2. ExternalTaskMarker reaches the states mentioned in the failed_states list
       In this case, ExternalTaskSensor will raise an AirflowException and user need to handle this
       with multiple downstream tasks
   3. ExternalTaskSensor times out
       In this case, ExternalTaskSensor will raise AirflowSkipException or AirflowSensorTimeout
       exception
   """
   
   import datetime
   
   from airflow import DAG
   from airflow.operators.dummy import DummyOperator
   from airflow.sensors.external_task import ExternalTaskMarker, ExternalTaskSensor
   
   start_date = datetime.datetime(2015, 1, 1)
   
   with DAG(
       dag_id="example_external_task_marker_parent",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as parent_dag:
       # [START howto_operator_external_task_marker]
       parent_task = ExternalTaskMarker(
           task_id="parent_task",
           external_dag_id="example_external_task_marker_child",
           external_task_id="child_task1",
       )
       # [END howto_operator_external_task_marker]
   
   with DAG(
       dag_id="example_external_task_marker_child",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as child_dag:
       # [START howto_operator_external_task_sensor]
       child_task1 = ExternalTaskSensor(
           task_id="child_task1",
           external_dag_id=parent_dag.dag_id,
           external_task_id=parent_task.task_id,
           timeout=600,
           allowed_states=['success'],
           failed_states=['failed', 'skipped'],
           mode="reschedule",
       )
       # [END howto_operator_external_task_sensor]
       child_task2 = DummyOperator(task_id="child_task2")
       child_task1 >> child_task2
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] hedrickw commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
hedrickw commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760526181


   @kaxil I also just wrote my own Sensor using the docs and BaseClass since you guys recommended that as well :D


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760531896


   The schedule_interval need to be the same for both DAGs for it to work and shouldn't be None.
   ```
   #
   # 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.
   
   """
   Example DAG demonstrating setting up inter-DAG dependencies using ExternalTaskSensor and
   ExternalTaskMarker
   
   In this example, child_task1 in example_external_task_marker_child depends on parent_task in
   example_external_task_marker_parent. When parent_task is cleared with "Recursive" selected,
   the presence of ExternalTaskMarker tells Airflow to clear child_task1 and its
   downstream tasks.
   
   ExternalTaskSensor will keep poking for the status of remote ExternalTaskMarker task at a regular
   interval till one of the following will happen:
   1. ExternalTaskMarker reaches the states mentioned in the allowed_states list
       In this case, ExternalTaskSensor will exit with a succes status code
   2. ExternalTaskMarker reaches the states mentioned in the failed_states list
       In this case, ExternalTaskSensor will raise an AirflowException and user need to handle this
       with multiple downstream tasks
   3. ExternalTaskSensor times out
       In this case, ExternalTaskSensor will raise AirflowSkipException or AirflowSensorTimeout
       exception
   """
   
   import datetime
   
   from airflow import DAG
   from airflow.operators.dummy import DummyOperator
   from airflow.sensors.external_task import ExternalTaskMarker, ExternalTaskSensor
   
   start_date = datetime.datetime(2015, 1, 1)
   
   with DAG(
       dag_id="example_external_task_marker_parent",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as parent_dag:
       # [START howto_operator_external_task_marker]
       parent_task = ExternalTaskMarker(
           task_id="parent_task",
           external_dag_id="example_external_task_marker_child",
           external_task_id="child_task1",
       )
       # [END howto_operator_external_task_marker]
   
   with DAG(
       dag_id="example_external_task_marker_child",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as child_dag:
       # [START howto_operator_external_task_sensor]
       child_task1 = ExternalTaskSensor(
           task_id="child_task1",
           external_dag_id=parent_dag.dag_id,
           external_task_id=parent_task.task_id,
           timeout=600,
           allowed_states=['success'],
           failed_states=['failed', 'skipped'],
           mode="reschedule",
       )
       # [END howto_operator_external_task_sensor]
       child_task2 = DummyOperator(task_id="child_task2")
       child_task1 >> child_task2
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] ephraimbuddy commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760531896


   The schedule_interval need to be the same for both DAGs for it to work.
   ```
   
   import datetime
   
   from airflow import DAG
   from airflow.operators.dummy import DummyOperator
   from airflow.sensors.external_task import ExternalTaskMarker, ExternalTaskSensor
   
   start_date = datetime.datetime(2015, 1, 1)
   
   with DAG(
       dag_id="example_external_task_marker_parent",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as parent_dag:
       # [START howto_operator_external_task_marker]
       parent_task = ExternalTaskMarker(
           task_id="parent_task",
           external_dag_id="example_external_task_marker_child",
           external_task_id="child_task1",
       )
       # [END howto_operator_external_task_marker]
   
   with DAG(
       dag_id="example_external_task_marker_child",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as child_dag:
       # [START howto_operator_external_task_sensor]
       child_task1 = ExternalTaskSensor(
           task_id="child_task1",
           external_dag_id=parent_dag.dag_id,
           external_task_id=parent_task.task_id,
           timeout=600,
           allowed_states=['success'],
           failed_states=['failed', 'skipped'],
           mode="reschedule",
       )
       # [END howto_operator_external_task_sensor]
       child_task2 = DummyOperator(task_id="child_task2")
       child_task1 >> child_task2
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] hedrickw commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
hedrickw commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760537641


   @ephraimbuddy Ah maybe its just an outdated file i grabbed example from master and it has schedule of None
   https://github.com/apache/airflow/blob/master/airflow/example_dags/example_external_task_marker_dag.py


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] hedrickw commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
hedrickw commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760525736


   @kaxil Ah so you need to Execute at the same time? I guess I manually triggered it from UI to try it out.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] eladkal commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
eladkal commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-896237921


   @ephraimbuddy is the issue here only fixing example dag ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hedrickw commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
hedrickw commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760539099


   @ephraimbuddy sure!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] ephraimbuddy commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-896240861


   > @ephraimbuddy is the issue here only fixing example dag ?
   
   Yes. @eladkal 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy edited a comment on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
ephraimbuddy edited a comment on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760531896


   The schedule_interval need to be the same for both DAGs for it to work.
   ```
   #
   # 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.
   
   """
   Example DAG demonstrating setting up inter-DAG dependencies using ExternalTaskSensor and
   ExternalTaskMarker
   
   In this example, child_task1 in example_external_task_marker_child depends on parent_task in
   example_external_task_marker_parent. When parent_task is cleared with "Recursive" selected,
   the presence of ExternalTaskMarker tells Airflow to clear child_task1 and its
   downstream tasks.
   
   ExternalTaskSensor will keep poking for the status of remote ExternalTaskMarker task at a regular
   interval till one of the following will happen:
   1. ExternalTaskMarker reaches the states mentioned in the allowed_states list
       In this case, ExternalTaskSensor will exit with a succes status code
   2. ExternalTaskMarker reaches the states mentioned in the failed_states list
       In this case, ExternalTaskSensor will raise an AirflowException and user need to handle this
       with multiple downstream tasks
   3. ExternalTaskSensor times out
       In this case, ExternalTaskSensor will raise AirflowSkipException or AirflowSensorTimeout
       exception
   """
   
   import datetime
   
   from airflow import DAG
   from airflow.operators.dummy import DummyOperator
   from airflow.sensors.external_task import ExternalTaskMarker, ExternalTaskSensor
   
   start_date = datetime.datetime(2015, 1, 1)
   
   with DAG(
       dag_id="example_external_task_marker_parent",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as parent_dag:
       # [START howto_operator_external_task_marker]
       parent_task = ExternalTaskMarker(
           task_id="parent_task",
           external_dag_id="example_external_task_marker_child",
           external_task_id="child_task1",
       )
       # [END howto_operator_external_task_marker]
   
   with DAG(
       dag_id="example_external_task_marker_child",
       start_date=start_date,
       schedule_interval=datetime.timedelta(minutes=30),
       tags=['example2'],
   ) as child_dag:
       # [START howto_operator_external_task_sensor]
       child_task1 = ExternalTaskSensor(
           task_id="child_task1",
           external_dag_id=parent_dag.dag_id,
           external_task_id=parent_task.task_id,
           timeout=600,
           allowed_states=['success'],
           failed_states=['failed', 'skipped'],
           mode="reschedule",
       )
       # [END howto_operator_external_task_sensor]
       child_task2 = DummyOperator(task_id="child_task2")
       child_task1 >> child_task2
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [airflow] kaxil commented on issue #13681: ExternalTaskSensor can never find External Parent Task

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #13681:
URL: https://github.com/apache/airflow/issues/13681#issuecomment-760516463


   Why did you manually trigger it?
   
   Secondly if you want to manually trigger it for trying it out, run both of them using cli with a specific execution date


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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