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/05/24 13:21:50 UTC

[GitHub] [airflow] vapiravfif opened a new issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

vapiravfif opened a new issue #16023:
URL: https://github.com/apache/airflow/issues/16023


   
   **Apache Airflow version**:
   2.0.2, 2.1.0
   
   **Kubernetes version (if you are using kubernetes)** (use `kubectl version`):
   Server Version: version.Info{Major:"1", Minor:"17+", GitVersion:"v1.17.12-eks-7684af", GitCommit:"7684af4ac41370dd109ac13817023cb8063e3d45", GitTreeState:"clean", BuildDate:"2020-10-20T22:57:40Z", GoVersion:"go1.13.15", Compiler:"gc", Platform:"linux/amd64"}
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**:
   AWS EKS
   - **Others**:
   Helm chart - 8.0.8, 8.1.0
   Executor - CeleryExecutor
   
   **What happened**:
   
   When DAG is paused, and long PythonOperator tasks triggered manually (with "Ingnore all deps" - "run"), they are failing with error:
   ```
   [2021-05-24 08:49:02,166] {logging_mixin.py:104} INFO - hi there, try 6, going to sleep for 15 secs
   [2021-05-24 08:49:03,808] {local_task_job.py:188} WARNING - State of this instance has been externally set to None. Terminating instance.
   [2021-05-24 08:49:03,810] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 172
   [2021-05-24 08:49:03,812] {taskinstance.py:1265} ERROR - Received SIGTERM. Terminating subprocesses.
   ```
   And in scheduler logs there’s message:
   ```
   [2021-05-24 08:48:59,471] {scheduler_job.py:1854} INFO - Resetting orphaned tasks for active dag runs
   [2021-05-24 08:48:59,485] {scheduler_job.py:1921} INFO - Reset the following 2 orphaned TaskInstances:
   	<TaskInstance: timeout_testing.run_param_all 2021-05-23 13:46:13.840235+00:00 [running]>
   	<TaskInstance: timeout_testing.sleep_well 2021-05-23 13:46:13.840235+00:00 [running]>
   ```
   
   **What you expected to happen**:
   
   These tasks are alive and well, and shouldn't be killed :)
   Looks like something in `reset_state_for_orphaned_tasks` is wrongly marking running tasks as abandoned...
   
   **How to reproduce it**:
   ```
   dag = DAG(os.path.basename(__file__).replace('.py', ''),
             start_date=datetime(2021, 5, 11),
             schedule_interval=timedelta(days=1))
   
   def sleep_tester(time_out, retries):
       for i in range(retries):
           print(f'hi there, try {i}, going to sleep for {time_out}')
           time.sleep(time_out)
           print("Aaah, good times, see ya soon")
   
   
   sleeping = PythonOperator(task_id="sleep_well",
                             python_callable=sleep_tester,
                             op_kwargs={'time_out': 15, 'retries': 50},
                             dag=dag)
   ```
   Create DAG with task above, verify it paused, trigger dag run manually from UI, then trigger the task manually. The task should fail after several tries.
   
   **Anything else we need to know**:
   It might happen only if DAG never was unpaused ("ON"), though couldn't verify 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.

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



[GitHub] [airflow] Jorricks commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   We are actually running into the same issue here as well.
   Sometimes our tasks aren't running long enough before they fail and don't get an external_executor_id. Then once we restart them in the webserver (when they don't have an `external_executor_id`), it still get's killed.


-- 
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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   > @Jorricks Would this be fixed by one of your PRs? (They haven't made it to a release yet.)
   
   Unfortunately not. My latest bugfix PR was to prevent scheduled tasks from being picked up and to make sure cleared tasks wouldn't be seen as orphans.
   
   
   I did some checks. I think it might be related to the fact that when running through the webserver, we never set the `external_executor_id` value. In a Scheduler run, this is set after a the task is queued, at which point the scheduler will read the event_buffer and set the `external_executor_id` based on that. However on the webserver side, this loop isn't present and `external_executor_id` isn't set, meaning when another scheduler picks it up, it dies.
   
   We should **not** see this behaviour for tasks that are first started by the Scheduler and then (re-)triggered manually.
   
   Code fragment of where we set it when tasks is started by the Scheduler:
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L596-L599  
   
   
   To verify my theory, it would help if you could provide DEBUG logs of the scheduler and the webserver while reproducing the issue. Could you please be so kind to provide these when you have the time?


-- 
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] SamWheating commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   We were experiencing a similar issue, but it turned out to be due to the SchedulerJob being marked as failed due to a slow heartbeat (due to adding a few thousand DAGs).
   
   Can you look for this line in your scheduler logs?
   `Marked %d SchedulerJob instances as failed`
   
   In our case, the solution was to increase the value of `scheduler.scheduler_health_check_threshold` in the config, which then prevented the schedulerJob from being killed. 
   
   Here's the part in the source code that deals with resetting tasks with missing SchedulerJobs:
   https://github.com/apache/airflow/blob/9c94b72d440b18a9e42123d20d48b951712038f9/airflow/jobs/scheduler_job.py#L1803


-- 
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] Jorricks commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   > @Jorricks Hi, it might be helpful to know that I couldn't reproduce the issue on a DAG that was previously active, but it reproduced like a charm on a brand new DAG with same setup. Both paused at the time of running the tasks.
   
   Yes this sort of goes hand in hand with what I expected. Once the scheduler kicked off tasks and you re-run them from the web interface, the tasks will keep their external_executor_id that you can't clear with Airflow 2.1.1. 
   Thanks for providing the logs and looking into 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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I did some checks. I think it might be related to the fact that when running through the webserver, we never set the `external_executor_id` value. In a Scheduler run, this is set after a the task is queued, at which point the scheduler will read the event_buffer and set the `external_executor_id` based on that. However on the webserver side, this loop isn't present and `external_executor_id` isn't set, meaning when another scheduler picks it up, it dies.
   
   We should **not** see this behaviour for tasks that are first started by the Scheduler and then (re-)triggered manually.
   
   Code fragment of where we set it when tasks is started by the Scheduler:
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L596-L599  
   
   
   To verify my theory, it would help if you could provide DEBUG logs of the scheduler and the webserver while reproducing the issue. Could you please be so kind to provide these when you have the time?


-- 
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] Lyalpha commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   Just to add that I'm also seeing this behaviour on 2.1.0, running on bare metal. Have a paused DAG and I am trying to run a task within the DAG manually, but upon the next check for orphaned tasks by the scheduler (on a 5min interval), the task is reset and receives a `SIGTERM`.
   
   No sign of `Marked %d SchedulerJob instances as failed` entries in logs. Given the original issue shows MWE, I don't see how it can be related to issues caused by a complex environment.


-- 
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 commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   Fixed in #19375, @vapiravfif can you test in 2.2.2 and reopen if it still happens


-- 
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] Jorricks commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   Is it possible you are restarting your webserver frequently?
   
   I think the problem is that it shouldn't go through this filter, but it somehow is.
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L1185
   
   Makes me wonder, what do you have for your [worker_refresh_interval](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#worker-refresh-interval) in the airflow.cfg?


-- 
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 #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   @Jorricks Would this be fixed by one of your PRs? (They haven't made it to a release 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] Jorricks commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   @ashb I don't mind creating a PR but do you have an idea on how we can fix this?
   I am having a bit of trouble figuring out how we can get the `external_executor_id` in the current flow without delaying the response of the request noticeably.


-- 
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] vapiravfif commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   Yep, not reproducing on 2.2.2, thank you!!


-- 
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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I think the problem is that it shouldn't go through this filter, but it somehow is.
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L1185
   
   Makes me wonder a couple of things:
   1. what do you have for your [worker_refresh_interval](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#worker-refresh-interval) in the airflow.cfg?
   2. Is it possible you are restarting your webserver frequently or after you started the task?
   3. To get to the root cause of the issue (and fix it) it would help if you could provide DEBUG logs of the scheduler while reproducing the issue. Could you please be so kind to provide these when you have the time?
   4. If you can't provide the full logs, do you see anything like the following in the logs (note that you should not search for %s as you won't find it) :
   a. "attempting to adopt pod %s"
   a. "Failed to adopt pod %s. Reason: %s" 
   b. "attempting to adopt task %s in dag %s which was not specified by database"
   


-- 
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] Jorricks removed a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

Posted by GitBox <gi...@apache.org>.
Jorricks removed a comment on issue #16023:
URL: https://github.com/apache/airflow/issues/16023#issuecomment-876729124


   I did some checks. I think it might be related to the fact that when running through the webserver, we never set the `external_executor_id` value. In a Scheduler run, this is set after a the task is queued, at which point the scheduler will read the event_buffer and set the `external_executor_id` based on that. However on the webserver side, this loop isn't present and `external_executor_id` isn't set, meaning when another scheduler picks it up, it dies.
   
   We should **not** see this behaviour for tasks that are first started by the Scheduler and then (re-)triggered manually.
   
   Code fragment of where we set it when tasks is started by the Scheduler:
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L596-L599  
   
   
   To verify my theory, it would help if you could provide DEBUG logs of the scheduler and the webserver while reproducing the issue. Could you please be so kind to provide these when you have the time?


-- 
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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I think the problem is that it shouldn't go through this filter, but it somehow is.
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L1185
   
   Makes me wonder two things:
   1. what do you have for your [worker_refresh_interval](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#worker-refresh-interval) in the airflow.cfg?
   2. Is it possible you are restarting your webserver frequently or after you started the task?
   


-- 
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] Jorricks commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   > @Jorricks Would this be fixed by one of your PRs? (They haven't made it to a release yet.)
   
   Unfortunately not. My latest bugfix PR was to prevent scheduled tasks from being picked up and to make sure cleared tasks wouldn't be seen as orphans.


-- 
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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I think the problem is that it shouldn't go through this filter, but it somehow is.
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L1185
   
   Makes me wonder a couple of things:
   1. what do you have for your [worker_refresh_interval](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#worker-refresh-interval) in the airflow.cfg?
   2. Is it possible you are restarting your webserver frequently or after you started the task?
   3. To get to the root cause of the issue (and fix it) it would help if you could provide DEBUG logs of the scheduler while reproducing the issue. Could you please be so kind to provide these when you have the time?


-- 
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] boring-cyborg[bot] commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #16023:
URL: https://github.com/apache/airflow/issues/16023#issuecomment-847038799


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


-- 
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 #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I ran the above provided DAG in LocalExecutor. Before I ran it, I did `airflow db reset`.
   The first 16 tasks were successful but the next 17 tasks failed. 
   In scheduler log, I found the following:
   ```
   [2021-08-20 21:31:41,056] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-04 00:00:00+00:00: scheduled__2021-06-04T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:41,144] {scheduler_job.py:1143} INFO - Resetting orphaned tasks for active dag runs
   [2021-08-20 21:31:41,152] {scheduler_job.py:1165} INFO - Marked 17 SchedulerJob instances as failed
   [2021-08-20 21:31:41,177] {scheduler_job.py:1209} INFO - Reset the following 10 orphaned TaskInstances:
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-05-27 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-05-28 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-02 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-03 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-06 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-07 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-08 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-09 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-10 00:00:00+00:00 [running]>
   	<TaskInstance: long_running_pyoperator.sleep_well 2021-06-11 00:00:00+00:00 [running]>
   [2021-08-20 21:31:42,225] {dag.py:2691} INFO - Setting next_dagrun for long_running_pyoperator to 2021-06-13T00:00:00+00:00
   [2021-08-20 21:31:42,307] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-06 00:00:00+00:00: scheduled__2021-06-06T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,324] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-07 00:00:00+00:00: scheduled__2021-06-07T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,333] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-08 00:00:00+00:00: scheduled__2021-06-08T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,352] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-09 00:00:00+00:00: scheduled__2021-06-09T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,374] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-10 00:00:00+00:00: scheduled__2021-06-10T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,383] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-11 00:00:00+00:00: scheduled__2021-06-11T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,400] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-05-27 00:00:00+00:00: scheduled__2021-05-27T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,422] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-05-28 00:00:00+00:00: scheduled__2021-05-28T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,429] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-05-29 00:00:00+00:00: scheduled__2021-05-29T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,443] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-05-30 00:00:00+00:00: scheduled__2021-05-30T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,463] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-05-31 00:00:00+00:00: scheduled__2021-05-31T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,484] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-01 00:00:00+00:00: scheduled__2021-06-01T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,503] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-02 00:00:00+00:00: scheduled__2021-06-02T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,511] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-03 00:00:00+00:00: scheduled__2021-06-03T00:00:00+00:00, externally triggered: False> failed
   [2021-08-20 21:31:42,519] {dagrun.py:431} ERROR - Marking run <DagRun long_running_pyoperator @ 2021-06-05 00:00:00+00:00: scheduled__2021-06-05T00:00:00+00:00, externally triggered: False> failed
   ```
   In the scheduler tasks log for dag processing, I found the following:
   
   ```log
   [2021-08-20 21:31:41,007] {processor.py:618} INFO - DAG(s) dict_keys(['long_running_pyoperator']) retrieved from /files/dags/long_running_pyoperator.py
   [2021-08-20 21:31:41,051] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-04 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,082] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-05-29 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,097] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-05 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,116] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-05-30 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,139] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-01 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,164] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-05-31 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,189] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-05-28 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,201] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-03 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,212] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-05-27 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,224] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-02 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,236] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-09 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,246] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-11 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,255] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-07 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,265] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-08 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,276] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-10 00:00:00+00:00 [failed]> in state failed
   [2021-08-20 21:31:41,285] {processor.py:575} INFO - Executed failure callback for <TaskInstance: long_running_pyoperator.sleep_well 2021-06-06 00:00:00+00:00 [failed]> in state failed
   ```
   That could be as a result of the schedulerJob being marked as failed.
   
   Then the task log:
   ```log
   [2021-08-20, 21:31:46 UTC] {local_task_job.py:209} WARNING - State of this instance has been externally set to failed. Terminating instance.
   [2021-08-20, 21:31:46 UTC] {process_utils.py:100} INFO - Sending Signals.SIGTERM to GPID 5738
   [2021-08-20, 21:31:46 UTC] {taskinstance.py:1369} ERROR - Received SIGTERM. Terminating subprocesses.
   [2021-08-20, 21:31:46 UTC] {process_utils.py:66} INFO - Process psutil.Process(pid=5738, status='terminated', exitcode=0, started='21:24:33') (5738) terminated with exit code 0
   ```
   


-- 
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 closed issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   


-- 
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] Jorricks edited a comment on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   I did some checks. I think it might be related to the fact that when running through the webserver, we never set the `external_executor_id` value. In a Scheduler run, this is set after a the task is queued, at which point the scheduler will read the event_buffer and set the `external_executor_id` based on that. However on the webserver side, this loop isn't present and `external_executor_id` isn't set, meaning when another scheduler picks it up, it dies.
   
   We should **not** see this behaviour for tasks that are first started by the Scheduler and then (re-)triggered manually.
   
   Code fragment of where we set it when tasks is started by the Scheduler.
   
   https://github.com/apache/airflow/blob/db6acd9e8a91e0eca9e12cace72edc57b2667d25/airflow/jobs/scheduler_job.py#L596-L599  
   
   
   To verify, I'm interested in these questions as well:
   1. what do you have for your [worker_refresh_interval](https://airflow.apache.org/docs/apache-airflow/stable/configurations-ref.html#worker-refresh-interval) in the airflow.cfg?
   2. Is it possible you are restarting your webserver frequently or after you started the task?
   3. To get to the root cause of the issue (and fix it) it would help if you could provide DEBUG logs of the scheduler while reproducing the issue. Could you please be so kind to provide these when you have the time?


-- 
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] vapiravfif commented on issue #16023: Running tasks marked as 'orphaned' and killed by scheduler

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


   @Jorricks Hi, it might be helpful to know that I couldn't reproduce the issue on a DAG that was previously active, but it reproduced like a charm on a brand new DAG with same setup. Both paused at the time of running the tasks.
   
   I can't find anything related to either `Setting external_id for` or `SchedulerJob instances as failed` in the logs for both instances (on DEBUG)
   
   Here's a `grep -A 5 -B 5` from logs related to the task name (there wasn't related logs on webserver...):
   ```
   Scheduler:
   [2021-07-11 08:28:08,961] {dag_processing.py:385} DEBUG - Received message of type DagParsingStat
   [2021-07-11 08:28:08,972] {scheduler_job.py:1854} INFO - Resetting orphaned tasks for active dag runs
   [2021-07-11 08:28:08,972] {scheduler_job.py:1862} DEBUG - Running SchedulerJob.adopt_or_reset_orphaned_tasks with retries. Try 1 of 3
   [2021-07-11 08:28:08,972] {scheduler_job.py:1864} DEBUG - Calling SchedulerJob.adopt_or_reset_orphaned_tasks method
   [2021-07-11 08:28:09,003] {scheduler_job.py:1921} INFO - Reset the following 1 orphaned TaskInstances:
       <TaskInstance: timeout_testing.sleep_operator_task 2021-07-11 08:02:31.293769+00:00 [running]>
   [2021-07-11 08:28:09,020] {scheduler_job.py:1399} DEBUG - Next timed event is in 0.645959
   [2021-07-11 08:28:09,020] {scheduler_job.py:1401} DEBUG - Ran scheduling loop in 0.13 seconds
   [2021-07-11 08:28:09,205] {settings.py:292} DEBUG - Disposing DB connection pool (PID 1178)
   [2021-07-11 08:28:09,223] {scheduler_job.py:310} DEBUG - Waiting for <ForkProcess(DagFileProcessor921-Process, stopped)>
   [2021-07-11 08:28:09,469] {settings.py:292} DEBUG - Disposing DB connection pool (PID 1182)
   
   
   Worker:
   [2021-07-11 08:23:47,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:23:52,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:23:57,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:02,558: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:07,132: INFO/MainProcess] Received task: airflow.executors.celery_executor.execute_command[ad28b4dc-6a78-4821-9a02-998aff8156b2]  
   [2021-07-11 08:24:07,133: DEBUG/MainProcess] TaskPool: Apply <function _fast_trace_task at 0x7feff474d3b0> (args:('airflow.executors.celery_executor.execute_command', 'ad28b4dc-6a78-4821-9a02-998aff8156b2', {'lang': 'py', 'task': 'airflow.executors.celery_executor.execute_command', 'id': 'ad28b4dc-6a78-4821-9a02-998aff8156b2', 'shadow': None, 'eta': None, 'expires': None, 'group': None, 'group_index': None, 'retries': 0, 'timelimit': [None, None], 'root_id': 'ad28b4dc-6a78-4821-9a02-998aff8156b2', 'parent_id': None, 'argsrepr': "[['airflow', 'tasks', 'run', 'timeout_testing', 'sleep_operator_task', '2021-07-11T08:02:31.293769+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/timeout_testing.py']]", 'kwargsrepr': '{}', 'origin': 'gen268@airflow-dev-web-6d79645c68-tzbnv', 'reply_to': 'a3aa91fd-479b-3d76-893e-7a8e8d23c454', 'correlation_id': 'ad28b4dc-6a78-4821-9a02-998aff8156b2', 'hostname': 'celery@airflow-dev-worker-0', 'delivery_info': {'exchange': '', 'routin
 g_key': 'default', 'priority': 0, 'redelivered': None}, 'args': [['airflow', 'tasks', 'run', 'timeout_testing', 'sleep_operator_task',... kwargs:{})
   [2021-07-11 08:24:07,134: DEBUG/MainProcess] Task accepted: airflow.executors.celery_executor.execute_command[ad28b4dc-6a78-4821-9a02-998aff8156b2] pid:45
   --
   [2021-07-11 08:24:07,134: DEBUG/MainProcess] Task accepted: airflow.executors.celery_executor.execute_command[ad28b4dc-6a78-4821-9a02-998aff8156b2] pid:45
   [2021-07-11 08:24:07,186: INFO/ForkPoolWorker-15] Executing command in Celery: ['airflow', 'tasks', 'run', 'timeout_testing', 'sleep_operator_task', '2021-07-11T08:02:31.293769+00:00', '--local', '--pool', 'default_pool', '--subdir', 'DAGS_FOLDER/timeout_testing.py']
   [2021-07-11 08:24:07,329: DEBUG/ForkPoolWorker-15] Calling callbacks: [<function default_action_log at 0x7ff0008deb90>]
   [2021-07-11 08:24:07,350: DEBUG/ForkPoolWorker-15] Setting up DB connection pool (PID 47)
   [2021-07-11 08:24:07,351: DEBUG/ForkPoolWorker-15] settings.prepare_engine_args(): Using NullPool
   [2021-07-11 08:24:07,353: INFO/ForkPoolWorker-15] Filling up the DagBag from ...dags_dev/timeout_testing.py
   [2021-07-11 08:24:07,354: DEBUG/ForkPoolWorker-15] Importing ...dags_dev/timeout_testing.py
   --
   --
   [2021-07-11 08:24:07,357: DEBUG/ForkPoolWorker-15] Loaded DAG <DAG: timeout_testing>
   [2021-07-11 08:24:07,399: DEBUG/ForkPoolWorker-15] Loading plugins
   [2021-07-11 08:24:07,399: DEBUG/ForkPoolWorker-15] Loading plugins from directory: /opt/airflow/plugins
   [2021-07-11 08:24:07,399: DEBUG/ForkPoolWorker-15] Loading plugins from entrypoints
   [2021-07-11 08:24:07,473: DEBUG/ForkPoolWorker-15] Integrate DAG plugins
   [2021-07-11 08:24:07,501: WARNING/ForkPoolWorker-15] Running <TaskInstance: timeout_testing.sleep_operator_task 2021-07-11T08:02:31.293769+00:00 [None]> on host airflow-dev-worker-0.airflow-dev-worker.airflow-dev.svc.cluster.local
   [2021-07-11 08:24:07,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:12,562: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:17,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:22,563: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   [2021-07-11 08:24:27,559: DEBUG/MainProcess] pidbox received method enable_events() [reply_to:None ticket:None]
   --
   ```
   
   I cannot share the full logs as I'm testing in real environment and there are many S3 requests that need hashing, but I saved the logs and happy to provide any part of them as needed. Thank you!


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