[
https://issues.apache.org/jira/browse/AIRFLOW-5071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17432762#comment-17432762
]
ASF GitHub Bot commented on AIRFLOW-5071:
-----------------------------------------
nguyenmphu commented on issue #10790:
URL: https://github.com/apache/airflow/issues/10790#issuecomment-949189802
I found that in the code of `airflow/jobs/scheduler_job.py`:
https://github.com/apache/airflow/blob/main/airflow/jobs/scheduler_job.py#L535
``` python
if ti.try_number == buffer_key.try_number and ti.state ==
State.QUEUED:
Stats.incr('scheduler.tasks.killed_externally')
msg = (
"Executor reports task instance %s finished (%s)
although the "
"task says its %s. (Info: %s) Was the task killed
externally?"
)
self.log.error(msg, ti, state, ti.state, info)
```
The scheduler checks the state of the task instance. When a task instance is
rescheduled (e.g: an external sensor), its state transition up_for_reschedule
-> scheduled -> queued -> running. If its state is queued and not moved to the
running state, the scheduler will raise an error.
So I think the code needs to change:
``` python
if ti.try_number == buffer_key.try_number and (
ti.state == State.QUEUED and not
TaskReschedule.find_for_task_instance(ti, session=session)
):
Stats.incr('scheduler.tasks.killed_externally')
msg = (
"Executor reports task instance %s finished (%s)
although the "
"task says its %s. (Info: %s) Was the task killed
externally?"
)
self.log.error(msg, ti, state, ti.state, info)
```
Here is my PR: https://github.com/apache/airflow/pull/19123
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> Thousand os Executor reports task instance X finished (success) although the
> task says its queued. Was the task killed externally?
> ----------------------------------------------------------------------------------------------------------------------------------
>
> Key: AIRFLOW-5071
> URL: https://issues.apache.org/jira/browse/AIRFLOW-5071
> Project: Apache Airflow
> Issue Type: Bug
> Components: DAG, scheduler
> Affects Versions: 1.10.3
> Reporter: msempere
> Priority: Critical
> Fix For: 1.10.12
>
> Attachments: image-2020-01-27-18-10-29-124.png,
> image-2020-07-08-07-58-42-972.png
>
>
> I'm opening this issue because since I update to 1.10.3 I'm seeing thousands
> of daily messages like the following in the logs:
>
> ```
> {{__init__.py:1580}} ERROR - Executor reports task instance <TaskInstance: X
> 2019-07-29 00:00:00+00:00 [queued]> finished (success) although the task says
> its queued. Was the task killed externally?
> {{jobs.py:1484}} ERROR - Executor reports task instance <TaskInstance: X
> 2019-07-29 00:00:00+00:00 [queued]> finished (success) although the task says
> its queued. Was the task killed externally?
> ```
> -And looks like this is triggering also thousand of daily emails because the
> flag to send email in case of failure is set to True.-
> I have Airflow setup to use Celery and Redis as a backend queue service.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)