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 2020/12/03 13:52:35 UTC

[GitHub] [airflow] turbaszek opened a new issue #12783: Sensors in reschedule mode are not rescheduled

turbaszek opened a new issue #12783:
URL: https://github.com/apache/airflow/issues/12783


   **Apache Airflow version**:
   2.0.0dev
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):
   
   **Environment**:
   ```
   ./breeze --python=3.8 --backend=postgres --db-reset restart
   ```
   
   **What happened**:
   
   Sensors in reschedule mode are not rescheduled by scheduler.
   
   **What you expected to happen**:
   
   Sensors in both poke and reschedule mode should work.
   
   **How to reproduce it**:
   
   ```
   from airflow import DAG
   from airflow.sensors.base_sensor_operator import BaseSensorOperator
   from airflow.utils.dates import days_ago
   
   class DummySensor(BaseSensorOperator):
       def poke(self, context):
           return False
   
   with DAG(
       "other_dag",
       start_date=days_ago(1),
       schedule_interval="*/5 * * * *",
       catchup=False
   ) as dag3:
       DummySensor(
           task_id='wait-task',
           poke_interval=60 * 5,
           mode='reschedule'
       )
   ```
   
   Unpause this DAG and see what's happening in webserver.
   
   The TI log:
   ```
   *** Reading local file: /root/airflow/logs/other_dag/wait-task/2020-12-03T13:45:00+00:00/1.log
   [2020-12-03 13:50:01,782] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: other_dag.wait-task 2020-12-03T13:45:00+00:00 [queued]>
   [2020-12-03 13:50:01,822] {taskinstance.py:826} INFO - Dependencies all met for <TaskInstance: other_dag.wait-task 2020-12-03T13:45:00+00:00 [queued]>
   [2020-12-03 13:50:01,826] {taskinstance.py:1017} INFO - 
   --------------------------------------------------------------------------------
   [2020-12-03 13:50:01,828] {taskinstance.py:1018} INFO - Starting attempt 1 of 1
   [2020-12-03 13:50:01,829] {taskinstance.py:1019} INFO - 
   --------------------------------------------------------------------------------
   [2020-12-03 13:50:01,852] {taskinstance.py:1038} INFO - Executing <Task(DummySensor): wait-task> on 2020-12-03T13:45:00+00:00
   [2020-12-03 13:50:01,865] {standard_task_runner.py:50} INFO - Started process 18758 to run task
   [2020-12-03 13:50:01,871] {standard_task_runner.py:74} INFO - Running: ['airflow', 'tasks', 'run', 'other_dag', 'wait-task', '2020-12-03T13:45:00+00:00', '--job-id', '7', '--pool', 'default_pool', '--raw', '--subdir', 'DAGS_FOLDER/the_old_issue.py', '--cfg-path', '/tmp/tmp5lbndq1_']
   [2020-12-03 13:50:01,874] {standard_task_runner.py:75} INFO - Job 7: Subtask wait-task
   [2020-12-03 13:50:02,115] {logging_mixin.py:103} INFO - Running <TaskInstance: other_dag.wait-task 2020-12-03T13:45:00+00:00 [running]> on host 053f6ca34e24
   [2020-12-03 13:50:02,239] {taskinstance.py:1230} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=airflow
   AIRFLOW_CTX_DAG_ID=other_dag
   AIRFLOW_CTX_TASK_ID=wait-task
   AIRFLOW_CTX_EXECUTION_DATE=2020-12-03T13:45:00+00:00
   AIRFLOW_CTX_DAG_RUN_ID=scheduled__2020-12-03T13:45:00+00:00
   [2020-12-03 13:50:02,299] {taskinstance.py:1386} INFO - Rescheduling task, marking task as UP_FOR_RESCHEDULE
   [2020-12-03 13:50:02,341] {local_task_job.py:118} INFO - Task exited with return code 0
   ```
   but instead of being put in up for reschedule mode the sensor as well as the dag run are failed.
   
   
   **Anything else we need to know**:
   
   Discovered when working on #10790 
   


----------------------------------------------------------------
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] turbaszek edited a comment on issue #12783: Sensors in reschedule mode are not rescheduled

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


   I can confirm that this issue does not exists in 1.10 branch and at ff1a2aa. Sensor is rescheduled as it should be.


----------------------------------------------------------------
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] ashb commented on issue #12783: Sensors in reschedule mode are not rescheduled

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


   I've worked out the problem -- it's caused by the switch to Serialized DAGs only in HA scheduler.
   
   I have a hacky-one line fix. and am working on a better fix now.


----------------------------------------------------------------
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] ashb closed issue #12783: Sensors in reschedule mode are not rescheduled

Posted by GitBox <gi...@apache.org>.
ashb closed issue #12783:
URL: https://github.com/apache/airflow/issues/12783


   


----------------------------------------------------------------
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] turbaszek commented on issue #12783: Sensors in reschedule mode are not rescheduled

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


   On of the problems is that scheduler schedule a task two times:
   ```
   [2020-12-03 15:23:03,332] {scheduler_job.py:1066} INFO - Setting the following tasks to queued state:
   	<TaskInstance: other_dag.wait-task 2020-12-03 15:22:00+00:00 [scheduled]>
   [2020-12-03 15:23:03,336] {scheduler_job.py:1108} INFO - Sending TaskInstanceKey(dag_id='other_dag', task_id='wait-task', execution_date=datetime.datetime(2020, 12, 3, 15, 22, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 1 and queue default
   [2020-12-03 15:23:03,337] {base_executor.py:78} DEBUG - TI: TaskInstanceKey(dag_id='other_dag', task_id='wait-task', execution_date=datetime.datetime(2020, 12, 3, 15, 22, tzinfo=Timezone('UTC')), try_number=1) QUEUED: OrderedDict(), RUNNING: {TaskInstanceKey(dag_id='other_dag', task_id='wait-task', execution_date=datetime.datetime(2020, 12, 3, 15, 22, tzinfo=Timezone('UTC')), try_number=1)}
   [2020-12-03 15:23:03,338] {base_executor.py:83} ERROR - Could not queue task TaskInstanceKey(dag_id='other_dag', task_id='wait-task', execution_date=datetime.datetime(2020, 12, 3, 15, 22, tzinfo=Timezone('UTC')), try_number=1)
   ```


----------------------------------------------------------------
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] turbaszek commented on issue #12783: Sensors in reschedule mode are not rescheduled

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


   I can confirm that this issue does not exists in 1.10 branch. Sensor is rescheduled as it should be.


----------------------------------------------------------------
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