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 2022/01/20 12:00:50 UTC

[GitHub] [airflow] nclaeys opened a new issue #20982: Trigger tasks from web-ui are killed by scheduler because they are orphaned task instances

nclaeys opened a new issue #20982:
URL: https://github.com/apache/airflow/issues/20982


   ### Apache Airflow version
   
   2.2.2
   
   ### What happened
   
   I have a dag which is scheduled every day and has depends_on_past set to true.
   When a task fails for a given date, which is expected as the required data is not there. If I want to manually run the next task (for the following day) the run always fails.
   
   The reason for this is that airflow creates next task instances for the following days and sets their state to None as they cannot be scheduled because the previous task instance is in failed state, this is correct and expected behavior. If I now manually trigger the task_run from the web-ui, the task_instance data in the database is not updated and as a consequence the queued_by_job_id is not filled in. 
   
   Every 5 minutes the airflow scheduler queries for orphaned tasks, and since my manual run does not have queued_by_job_id filled in, it always gets killed as the scheduler thinks it is orphaned.
   The scheduler shows the following logs:
   `
   airflow-scheduler [2022-01-20 11:15:41,784] {scheduler_job.py:1178} INFO - Reset the following 1 orphaned TaskInstances:                                                                   airflow-scheduler     <TaskInstance: testmanual.sample scheduled__2022-01-16T00:00:00+00:00 [running]> 
   `
   
   ### What you expected to happen
   
   I expect that the manual run will not be killed by the scheduler as it thinks it is orphaned and thus that my tasks can still succeed.
   
   If this is expected behavior, It is best to show an error to the user when it tries to run the request stating: Running this task manually is not supported because... Then at least it is clear to the user, now the actual reason is really hidden and not obvious for most users I assume. 
   
   ### How to reproduce
   
   - Create a dag that fails for the start_date and for the following dates just sleeps for 5 minutes and a start date to yesterday.
   - Enable the dag and the run for yesterday will fail (expected)
   - Manually trigger the dag for today through the web-ui
   - The run for today will be killed by the scheduler
   
   ### Operating System
   
   debian buster
   
   ### Versions of Apache Airflow Providers
   
   /
   
   ### Deployment
   
   Other Docker-based deployment
   
   ### Deployment details
   
   We run airflow on kubernetes and thus use the kubernetes_executor to schedule tasks.
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
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] nclaeys commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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






-- 
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] nclaeys commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does:
   `
   session.query(TI)
                           .filter(TI.state.in_(resettable_states))
                           .outerjoin(TI.queued_by_job)
                           .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING))
                           .join(TI.dag_run)
                           .filter(
                               DagRun.run_type != DagRunType.BACKFILL_JOB,
                               DagRun.state == State.RUNNING,
                           )
   `
   and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.


-- 
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] potiuk commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   Do you want to attempt to fix it @nclaeys ?


-- 
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] ashb commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   I do remember we already made some changes in the orphan code, so there is a _chance_ this has already been fixed.


-- 
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] potiuk commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   @ashb @uranusjr @ephraimbuddy @jedcunningham  - I marked it for 2.2.4 - I've seen this one reported by others (I think so: https://github.com/apache/airflow/discussions/21047 ). I guess it might be a bad query somewhere in orphan detection which does not exclude the tasks that are run through other means from adoption by scheduler.
   
   I might be wrong about it and I might take a look shortly, but maybe you know something that I would have to find out through digging in the code.


-- 
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] potiuk edited a comment on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   > @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does: `session.query(TI) .filter(TI.state.in_(resettable_states)) .outerjoin(TI.queued_by_job) .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING)) .join(TI.dag_run) .filter( DagRun.run_type != DagRunType.BACKFILL_JOB, DagRun.state == State.RUNNING, )` and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.
   
   I think it ain't so easy. Unlike backfills (controlled by the client `airflow backfill` command) , the manual runs are also "controlled" by scheduler - same as regular runs. That's why backfills are excluded in this query.
   
    I have a feeling this is more a problem with kubernetes_executor's adoption code which uses labels: https://github.com/apache/airflow/blob/1169e3a81f5481e92d29e174163eb5e6d7f3fe93/airflow/executors/kubernetes_executor.py#L678
   
   But this is more of an intuition than looking at the code yet.


-- 
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] potiuk edited a comment on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   > @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does: `session.query(TI) .filter(TI.state.in_(resettable_states)) .outerjoin(TI.queued_by_job) .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING)) .join(TI.dag_run) .filter( DagRun.run_type != DagRunType.BACKFILL_JOB, DagRun.state == State.RUNNING, )` and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.
   
   I think it ain't so easy. Unlike backfills (controlled by the client `airflow backfill` command) , the manual runs are also "controlled" by scheduler - same as regular runs. That's why backfills are excluded in this query.
   
    I have a feeling this is more a problem with kubernetes_executor's adoption code which uses labels: https://github.com/apache/airflow/blob/1169e3a81f5481e92d29e174163eb5e6d7f3fe93/airflow/executors/kubernetes_executor.py#L678
   
   But this is more of an intuition than looking at the code yet.


-- 
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] potiuk edited a comment on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   @ashb @uranusjr @ephraimbuddy @jedcunningham  - I marked it for 2.2.4 - I've seen this one reported by others (I think so: https://github.com/apache/airflow/discussions/21047 ). I guess it might be a bad query somewhere in orphan detection which does not exclude the tasks that are run through other means from adoption by scheduler.
   
   I might be wrong about it and I might take a look shortly, but maybe you know something that I would have to find out through digging in the code.
   
   And I think this one is pretty disruptive as well.


-- 
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] nclaeys commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   Well I do not think the problem is in the adopt_task_instances but I am not sure what the expected behavior should be. It only tries adopting task instances that have the queued_by_job_id field filled in, in which case it changes the labels identifying the previous scheduler pod name with the new one in order to watch them (makes sense in my opinion). In the manually triggered run, we do not have a queued_by_job_id so it is not adopted.
   
   If I am not mistaken the run button triggers a local execution of the task within the web pod and creates a local kubernetes executor to launch the task, which is different than a task that is triggered from the scheduler pod itself. 
   
   I am not sure what the expected behavior should be:
   - we could change the query to find orphaned tasks
   - we should make sure the queued_by_job_id is filled in even if it is a local execution of the job? But in that case we probably do not want it to be adopted by the scheduler pod as it is already managed by the web pod...


-- 
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] potiuk commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   > @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does: `session.query(TI) .filter(TI.state.in_(resettable_states)) .outerjoin(TI.queued_by_job) .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING)) .join(TI.dag_run) .filter( DagRun.run_type != DagRunType.BACKFILL_JOB, DagRun.state == State.RUNNING, )` and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.
   
   I think it ain't so easy. Unlike Backfills Manual runs are also "controlled" by scheduler. I have a feeling this is more a problem with kubernetes_executor's adoption code which uses labels: https://github.com/apache/airflow/blob/1169e3a81f5481e92d29e174163eb5e6d7f3fe93/airflow/executors/kubernetes_executor.py#L678
   
   But this is more of an intuition than looking at the code yet.


-- 
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] potiuk commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   It does seam like pretty unintended behaviour.


-- 
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] potiuk commented on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   > @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does: `session.query(TI) .filter(TI.state.in_(resettable_states)) .outerjoin(TI.queued_by_job) .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING)) .join(TI.dag_run) .filter( DagRun.run_type != DagRunType.BACKFILL_JOB, DagRun.state == State.RUNNING, )` and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.
   
   I think it ain't so easy. Unlike Backfills Manual runs are also "controlled" by scheduler. I have a feeling this is more a problem with kubernetes_executor's adoption code which uses labels: https://github.com/apache/airflow/blob/1169e3a81f5481e92d29e174163eb5e6d7f3fe93/airflow/executors/kubernetes_executor.py#L678
   
   But this is more of an intuition than looking at the code yet.


-- 
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] nclaeys edited a comment on issue #20982: Triggered tasks from web-ui are killed by scheduler because they are orphaned task instances

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


   @potiuk I will take a look at it. I am not that familiar with the inner workings of airflow, but I think that the query for orphaned tasks is wrong. At the moment it does:
   ```
   session.query(TI)
                           .filter(TI.state.in_(resettable_states))
                           .outerjoin(TI.queued_by_job)
                           .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING))
                           .join(TI.dag_run)
                           .filter(
                               DagRun.run_type != DagRunType.BACKFILL_JOB,
                               DagRun.state == State.RUNNING,
                           )
   ```
   and I think that in my case the DagRun.run_type == DagRunType.Manual. I will validate this and supply a fix for it.


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