[
https://issues.apache.org/jira/browse/AIRFLOW-5071?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17232494#comment-17232494
]
ASF GitHub Bot commented on AIRFLOW-5071:
-----------------------------------------
freedom1989 commented on issue #10790:
URL: https://github.com/apache/airflow/issues/10790#issuecomment-727719402
The cause is clear as @rafalkozik mentioned. After scheduler schedule the
task at the second time(put it in queue) and then it start process the executor
events of the task's first-try. It occurs when the scheduling loop time >
sensor task reschedule interval.
Either reducing the scheduler looping time(dag processing time, etc) or
increasing the sensor task reschedule interval will work.
The bug can also be fixed if the rescheduled task instance use a different
try number, but this will cause a lot of log files.
```
def _process_executor_events(self, simple_dag_bag, session=None):
# ...
if ti.try_number == try_number and ti.state == State.QUEUED:
# <-- try number for a sensor task is always the same
msg = ("Executor reports task instance {} finished ({}) "
"although the task says its {}. Was the task "
"killed externally?".format(ti, state, ti.state))
Stats.incr('scheduler.tasks.killed_externally')
self.log.error(msg)
try:
simple_dag = simple_dag_bag.get_dag(dag_id)
dagbag = models.DagBag(simple_dag.full_filepath)
dag = dagbag.get_dag(dag_id)
ti.task = dag.get_task(task_id)
ti.handle_failure(msg)
except Exception:
self.log.error("Cannot load the dag bag to handle
failure for %s"
". Setting task to FAILED without
callbacks or "
"retries. Do you have enough
resources?", ti)
ti.state = State.FAILED
session.merge(ti)
session.commit()
```
----------------------------------------------------------------
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:
[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)