You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@airflow.apache.org by Bolke de Bruin <bd...@gmail.com> on 2016/09/04 13:34:05 UTC

Scheduler getting stuck - request for details

Hi All,

We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details. 

First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following

- Airflow version (pip show airflow)
- Celery version (pip show celery)
- Kombu version (pip show kombu)

- Redis version (if applicable)
- RabbitMQ version (if applicable)

- Sanitized airflow configuration
- Sanitized broker configuration

If possible supply, preferably debug, logs of broker, scheduler and worker. 

Thanks!
Bolke


Re: Scheduler getting stuck - request for details

Posted by הילה ויזן <hi...@gmail.com>.
Hi Jeff & Bolke,

Did you find anything related to this issue?
We tried to wrote a script that restarts the scheduler each 30 minutes, but
we don't want to use this patch.
We really want to understand the root cause of the problem, and understand
if there will be an official fix.


On Thu, Sep 8, 2016 at 12:28 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> Furthermore have you tried it as well with RabbitMQ as Redis does not
> guarantee delivery (neither with Celery)?
> Is your redis setup located on the same machine as the scheduler or
> separated?
>
> Thanks!
> Bolke
>
> > Op 7 sep. 2016, om 23:17 heeft Bolke de Bruin <bd...@gmail.com> het
> volgende geschreven:
> >
> >
> >> Op 7 sep. 2016, om 21:37 heeft Jeff Balogh <jb...@stripe.com.INVALID>
> het volgende geschreven:
> >>
> >> On Wed, Sep 7, 2016 at 12:17 PM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>> Ah this is the more interesting case. Are you getting tasks into
> SCHEDULED and then the scheduler itself gets stuck? Or do the workers not
> execute anything anymore?
> >>
> >> The tasks are put into the SCHEDULED state but they don't make it to a
> >> worker. This isn't deterministic. With our patch to clean up orphans,
> >> a task may flap in SCHEDULED a few times but eventually it makes it to
> >> a worker.
> >
> > Ok. So are you implying that the executor is not picking up the tasks or
> that the queue is losing tasks? Are you able to find out what redis is
> doing when a ’scheduled’ task is flapping, ie does it receive the task at
> all? Btw what happened before having the scheduled state in?
> >
> >>
> >> The scheduler and workers are otherwise running fine. We've been
> >> running with the same celery/redis setup for a year.
> >>
> >>> How do you run your scheduler? With num_runs?
> >>
> >> We don't use num_runs. We restart the scheduler when we deploy new code.
> >>
> >>> A later patch checks for these “orphaned_tasks” at scheduler start up.
> >>
> >> We check for the orphans at the top of the scheduler loop, so on every
> run.
> >
> > Ok we moved away from this for performance reasons. Depending on a
> solution for the above issue we might need to apply it to every run then.
> >
> >>
> >>> In other words can you provide some more information :-).
> >>>
> >>> Bolke
> >>>
> >>>> Op 7 sep. 2016, om 20:08 heeft Jeff Balogh <jb...@stripe.com.INVALID>
> het volgende geschreven:
> >>>>
> >>>> Ah yep, we're on https://github.com/apache/incubator-airflow/commits/
> 54b361d2a.
> >>>>
> >>>> On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>>>> Hi Jeff,
> >>>>>
> >>>>> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was
> introduced after release. Are you sure you are on 1.7.1.3 and not on master?
> >>>>>
> >>>>> Bolke
> >>>>>
> >>>>>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh
> <jb...@stripe.com.INVALID> het volgende geschreven:
> >>>>>>
> >>>>>> When we bumped to 1.7.1.3 we found that tasks would go into the new
> >>>>>> SCHEDULED state and get stuck there. We haven't determined why this
> >>>>>> happens.
> >>>>>>
> >>>>>> We put a hacky patch into our scheduler that sets state to None for
> >>>>>> any tasks that are SCHEDULED at the beginning of the schedule loop.
> >>>>>>
> >>>>>> Name: airflow
> >>>>>> Version: 1.7.1.3
> >>>>>> Name: celery
> >>>>>> Version: 3.1.23
> >>>>>> Name: kombu
> >>>>>> Version: 3.0.35
> >>>>>>
> >>>>>> redis_version:2.6.13
> >>>>>>
> >>>>>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>>>>>> Hi All,
> >>>>>>>
> >>>>>>> We have had some reports on this list and sometimes on Jira that
> the scheduler sometimes seems to get stuck. I would like to track down this
> issue, but until now much of the reporting has been a bit light on the
> details.
> >>>>>>>
> >>>>>>> First and foremost I am assuming that getting “stuck” is only
> happening when using a CeleryExecutor. To further track down the issue I
> would like to know the following
> >>>>>>>
> >>>>>>> - Airflow version (pip show airflow)
> >>>>>>> - Celery version (pip show celery)
> >>>>>>> - Kombu version (pip show kombu)
> >>>>>>>
> >>>>>>> - Redis version (if applicable)
> >>>>>>> - RabbitMQ version (if applicable)
> >>>>>>>
> >>>>>>> - Sanitized airflow configuration
> >>>>>>> - Sanitized broker configuration
> >>>>>>>
> >>>>>>> If possible supply, preferably debug, logs of broker, scheduler
> and worker.
> >>>>>>>
> >>>>>>> Thanks!
> >>>>>>> Bolke
> >>>>>>>
> >>>>>
> >>>
> >
>
>

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
Furthermore have you tried it as well with RabbitMQ as Redis does not guarantee delivery (neither with Celery)?
Is your redis setup located on the same machine as the scheduler or separated?

Thanks!
Bolke

> Op 7 sep. 2016, om 23:17 heeft Bolke de Bruin <bd...@gmail.com> het volgende geschreven:
> 
> 
>> Op 7 sep. 2016, om 21:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>> 
>> On Wed, Sep 7, 2016 at 12:17 PM, Bolke de Bruin <bd...@gmail.com> wrote:
>>> Ah this is the more interesting case. Are you getting tasks into SCHEDULED and then the scheduler itself gets stuck? Or do the workers not execute anything anymore?
>> 
>> The tasks are put into the SCHEDULED state but they don't make it to a
>> worker. This isn't deterministic. With our patch to clean up orphans,
>> a task may flap in SCHEDULED a few times but eventually it makes it to
>> a worker.
> 
> Ok. So are you implying that the executor is not picking up the tasks or that the queue is losing tasks? Are you able to find out what redis is doing when a ’scheduled’ task is flapping, ie does it receive the task at all? Btw what happened before having the scheduled state in?
> 
>> 
>> The scheduler and workers are otherwise running fine. We've been
>> running with the same celery/redis setup for a year.
>> 
>>> How do you run your scheduler? With num_runs?
>> 
>> We don't use num_runs. We restart the scheduler when we deploy new code.
>> 
>>> A later patch checks for these “orphaned_tasks” at scheduler start up.
>> 
>> We check for the orphans at the top of the scheduler loop, so on every run.
> 
> Ok we moved away from this for performance reasons. Depending on a solution for the above issue we might need to apply it to every run then.
> 
>> 
>>> In other words can you provide some more information :-).
>>> 
>>> Bolke
>>> 
>>>> Op 7 sep. 2016, om 20:08 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>>> 
>>>> Ah yep, we're on https://github.com/apache/incubator-airflow/commits/54b361d2a.
>>>> 
>>>> On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>> Hi Jeff,
>>>>> 
>>>>> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?
>>>>> 
>>>>> Bolke
>>>>> 
>>>>>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>>>>> 
>>>>>> When we bumped to 1.7.1.3 we found that tasks would go into the new
>>>>>> SCHEDULED state and get stuck there. We haven't determined why this
>>>>>> happens.
>>>>>> 
>>>>>> We put a hacky patch into our scheduler that sets state to None for
>>>>>> any tasks that are SCHEDULED at the beginning of the schedule loop.
>>>>>> 
>>>>>> Name: airflow
>>>>>> Version: 1.7.1.3
>>>>>> Name: celery
>>>>>> Version: 3.1.23
>>>>>> Name: kombu
>>>>>> Version: 3.0.35
>>>>>> 
>>>>>> redis_version:2.6.13
>>>>>> 
>>>>>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>>>> Hi All,
>>>>>>> 
>>>>>>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>>>>>>> 
>>>>>>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>>>>>>> 
>>>>>>> - Airflow version (pip show airflow)
>>>>>>> - Celery version (pip show celery)
>>>>>>> - Kombu version (pip show kombu)
>>>>>>> 
>>>>>>> - Redis version (if applicable)
>>>>>>> - RabbitMQ version (if applicable)
>>>>>>> 
>>>>>>> - Sanitized airflow configuration
>>>>>>> - Sanitized broker configuration
>>>>>>> 
>>>>>>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>>>>>>> 
>>>>>>> Thanks!
>>>>>>> Bolke
>>>>>>> 
>>>>> 
>>> 
> 


Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
> Op 7 sep. 2016, om 21:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
> 
> On Wed, Sep 7, 2016 at 12:17 PM, Bolke de Bruin <bd...@gmail.com> wrote:
>> Ah this is the more interesting case. Are you getting tasks into SCHEDULED and then the scheduler itself gets stuck? Or do the workers not execute anything anymore?
> 
> The tasks are put into the SCHEDULED state but they don't make it to a
> worker. This isn't deterministic. With our patch to clean up orphans,
> a task may flap in SCHEDULED a few times but eventually it makes it to
> a worker.

Ok. So are you implying that the executor is not picking up the tasks or that the queue is losing tasks? Are you able to find out what redis is doing when a ’scheduled’ task is flapping, ie does it receive the task at all? Btw what happened before having the scheduled state in?

> 
> The scheduler and workers are otherwise running fine. We've been
> running with the same celery/redis setup for a year.
> 
>> How do you run your scheduler? With num_runs?
> 
> We don't use num_runs. We restart the scheduler when we deploy new code.
> 
>> A later patch checks for these “orphaned_tasks” at scheduler start up.
> 
> We check for the orphans at the top of the scheduler loop, so on every run.

Ok we moved away from this for performance reasons. Depending on a solution for the above issue we might need to apply it to every run then.

> 
>> In other words can you provide some more information :-).
>> 
>> Bolke
>> 
>>> Op 7 sep. 2016, om 20:08 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>> 
>>> Ah yep, we're on https://github.com/apache/incubator-airflow/commits/54b361d2a.
>>> 
>>> On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>> Hi Jeff,
>>>> 
>>>> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?
>>>> 
>>>> Bolke
>>>> 
>>>>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>>>> 
>>>>> When we bumped to 1.7.1.3 we found that tasks would go into the new
>>>>> SCHEDULED state and get stuck there. We haven't determined why this
>>>>> happens.
>>>>> 
>>>>> We put a hacky patch into our scheduler that sets state to None for
>>>>> any tasks that are SCHEDULED at the beginning of the schedule loop.
>>>>> 
>>>>> Name: airflow
>>>>> Version: 1.7.1.3
>>>>> Name: celery
>>>>> Version: 3.1.23
>>>>> Name: kombu
>>>>> Version: 3.0.35
>>>>> 
>>>>> redis_version:2.6.13
>>>>> 
>>>>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>>> Hi All,
>>>>>> 
>>>>>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>>>>>> 
>>>>>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>>>>>> 
>>>>>> - Airflow version (pip show airflow)
>>>>>> - Celery version (pip show celery)
>>>>>> - Kombu version (pip show kombu)
>>>>>> 
>>>>>> - Redis version (if applicable)
>>>>>> - RabbitMQ version (if applicable)
>>>>>> 
>>>>>> - Sanitized airflow configuration
>>>>>> - Sanitized broker configuration
>>>>>> 
>>>>>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>>>>>> 
>>>>>> Thanks!
>>>>>> Bolke
>>>>>> 
>>>> 
>> 


Re: Scheduler getting stuck - request for details

Posted by Jeff Balogh <jb...@stripe.com.INVALID>.
On Wed, Sep 7, 2016 at 12:17 PM, Bolke de Bruin <bd...@gmail.com> wrote:
> Ah this is the more interesting case. Are you getting tasks into SCHEDULED and then the scheduler itself gets stuck? Or do the workers not execute anything anymore?

The tasks are put into the SCHEDULED state but they don't make it to a
worker. This isn't deterministic. With our patch to clean up orphans,
a task may flap in SCHEDULED a few times but eventually it makes it to
a worker.

The scheduler and workers are otherwise running fine. We've been
running with the same celery/redis setup for a year.

> How do you run your scheduler? With num_runs?

We don't use num_runs. We restart the scheduler when we deploy new code.

> A later patch checks for these “orphaned_tasks” at scheduler start up.

We check for the orphans at the top of the scheduler loop, so on every run.

> In other words can you provide some more information :-).
>
> Bolke
>
>> Op 7 sep. 2016, om 20:08 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>
>> Ah yep, we're on https://github.com/apache/incubator-airflow/commits/54b361d2a.
>>
>> On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>> Hi Jeff,
>>>
>>> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?
>>>
>>> Bolke
>>>
>>>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>>>
>>>> When we bumped to 1.7.1.3 we found that tasks would go into the new
>>>> SCHEDULED state and get stuck there. We haven't determined why this
>>>> happens.
>>>>
>>>> We put a hacky patch into our scheduler that sets state to None for
>>>> any tasks that are SCHEDULED at the beginning of the schedule loop.
>>>>
>>>> Name: airflow
>>>> Version: 1.7.1.3
>>>> Name: celery
>>>> Version: 3.1.23
>>>> Name: kombu
>>>> Version: 3.0.35
>>>>
>>>> redis_version:2.6.13
>>>>
>>>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>> Hi All,
>>>>>
>>>>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>>>>>
>>>>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>>>>>
>>>>> - Airflow version (pip show airflow)
>>>>> - Celery version (pip show celery)
>>>>> - Kombu version (pip show kombu)
>>>>>
>>>>> - Redis version (if applicable)
>>>>> - RabbitMQ version (if applicable)
>>>>>
>>>>> - Sanitized airflow configuration
>>>>> - Sanitized broker configuration
>>>>>
>>>>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>>>>>
>>>>> Thanks!
>>>>> Bolke
>>>>>
>>>
>

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
Ah this is the more interesting case. Are you getting tasks into SCHEDULED and then the scheduler itself gets stuck? Or do the workers not execute anything anymore? 
How do you run your scheduler? With num_runs? A later patch checks for these “orphaned_tasks” at scheduler start up.

In other words can you provide some more information :-).

Bolke

> Op 7 sep. 2016, om 20:08 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
> 
> Ah yep, we're on https://github.com/apache/incubator-airflow/commits/54b361d2a.
> 
> On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>> Hi Jeff,
>> 
>> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?
>> 
>> Bolke
>> 
>>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>> 
>>> When we bumped to 1.7.1.3 we found that tasks would go into the new
>>> SCHEDULED state and get stuck there. We haven't determined why this
>>> happens.
>>> 
>>> We put a hacky patch into our scheduler that sets state to None for
>>> any tasks that are SCHEDULED at the beginning of the schedule loop.
>>> 
>>> Name: airflow
>>> Version: 1.7.1.3
>>> Name: celery
>>> Version: 3.1.23
>>> Name: kombu
>>> Version: 3.0.35
>>> 
>>> redis_version:2.6.13
>>> 
>>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>>> Hi All,
>>>> 
>>>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>>>> 
>>>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>>>> 
>>>> - Airflow version (pip show airflow)
>>>> - Celery version (pip show celery)
>>>> - Kombu version (pip show kombu)
>>>> 
>>>> - Redis version (if applicable)
>>>> - RabbitMQ version (if applicable)
>>>> 
>>>> - Sanitized airflow configuration
>>>> - Sanitized broker configuration
>>>> 
>>>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>>>> 
>>>> Thanks!
>>>> Bolke
>>>> 
>> 


Re: Scheduler getting stuck - request for details

Posted by Jeff Balogh <jb...@stripe.com.INVALID>.
Ah yep, we're on https://github.com/apache/incubator-airflow/commits/54b361d2a.

On Wed, Sep 7, 2016 at 10:13 AM, Bolke de Bruin <bd...@gmail.com> wrote:
> Hi Jeff,
>
> That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?
>
> Bolke
>
>> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
>>
>> When we bumped to 1.7.1.3 we found that tasks would go into the new
>> SCHEDULED state and get stuck there. We haven't determined why this
>> happens.
>>
>> We put a hacky patch into our scheduler that sets state to None for
>> any tasks that are SCHEDULED at the beginning of the schedule loop.
>>
>> Name: airflow
>> Version: 1.7.1.3
>> Name: celery
>> Version: 3.1.23
>> Name: kombu
>> Version: 3.0.35
>>
>> redis_version:2.6.13
>>
>> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>> Hi All,
>>>
>>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>>>
>>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>>>
>>> - Airflow version (pip show airflow)
>>> - Celery version (pip show celery)
>>> - Kombu version (pip show kombu)
>>>
>>> - Redis version (if applicable)
>>> - RabbitMQ version (if applicable)
>>>
>>> - Sanitized airflow configuration
>>> - Sanitized broker configuration
>>>
>>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>>>
>>> Thanks!
>>> Bolke
>>>
>

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
Hi Jeff,

That is kind of impossible for 1.7.1.3 as the SCHEDULED state was introduced after release. Are you sure you are on 1.7.1.3 and not on master?

Bolke

> Op 7 sep. 2016, om 18:37 heeft Jeff Balogh <jb...@stripe.com.INVALID> het volgende geschreven:
> 
> When we bumped to 1.7.1.3 we found that tasks would go into the new
> SCHEDULED state and get stuck there. We haven't determined why this
> happens.
> 
> We put a hacky patch into our scheduler that sets state to None for
> any tasks that are SCHEDULED at the beginning of the schedule loop.
> 
> Name: airflow
> Version: 1.7.1.3
> Name: celery
> Version: 3.1.23
> Name: kombu
> Version: 3.0.35
> 
> redis_version:2.6.13
> 
> On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>> Hi All,
>> 
>> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>> 
>> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>> 
>> - Airflow version (pip show airflow)
>> - Celery version (pip show celery)
>> - Kombu version (pip show kombu)
>> 
>> - Redis version (if applicable)
>> - RabbitMQ version (if applicable)
>> 
>> - Sanitized airflow configuration
>> - Sanitized broker configuration
>> 
>> If possible supply, preferably debug, logs of broker, scheduler and worker.
>> 
>> Thanks!
>> Bolke
>> 


Re: Scheduler getting stuck - request for details

Posted by Jeff Balogh <jb...@stripe.com.INVALID>.
When we bumped to 1.7.1.3 we found that tasks would go into the new
SCHEDULED state and get stuck there. We haven't determined why this
happens.

We put a hacky patch into our scheduler that sets state to None for
any tasks that are SCHEDULED at the beginning of the schedule loop.

Name: airflow
Version: 1.7.1.3
Name: celery
Version: 3.1.23
Name: kombu
Version: 3.0.35

redis_version:2.6.13

On Sun, Sep 4, 2016 at 6:34 AM, Bolke de Bruin <bd...@gmail.com> wrote:
> Hi All,
>
> We have had some reports on this list and sometimes on Jira that the scheduler sometimes seems to get stuck. I would like to track down this issue, but until now much of the reporting has been a bit light on the details.
>
> First and foremost I am assuming that getting “stuck” is only happening when using a CeleryExecutor. To further track down the issue I would like to know the following
>
> - Airflow version (pip show airflow)
> - Celery version (pip show celery)
> - Kombu version (pip show kombu)
>
> - Redis version (if applicable)
> - RabbitMQ version (if applicable)
>
> - Sanitized airflow configuration
> - Sanitized broker configuration
>
> If possible supply, preferably debug, logs of broker, scheduler and worker.
>
> Thanks!
> Bolke
>

Re: Scheduler getting stuck - request for details

Posted by Gerard Toonstra <gt...@gmail.com>.
The scheduler is probably single threaded, but it's a good idea to make
sure and investigate postgres (or mysql) locks:

https://wiki.postgresql.org/wiki/Lock_Monitoring

On Wed, Sep 7, 2016 at 8:30 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> Thanks!
>
> Apache scrubs attachments. Can you please put it somewhere where it can be
> downloaded (pastebin).
>
> Full logs are more helpful than just the end. If those can be downloaded
> that would be nice.
>
> Bolke
>
>
> > Op 7 sep. 2016, om 08:24 heeft הילה ויזן <hi...@gmail.com> het
> volgende geschreven:
> >
> > Hi,
> > we face the same issue with latest version.
> > environment:
> > airflow 1.7.1.3.
> > postgress 9.2.13 (backend DB)
> > OS   Red Hat Enterprise Linux Server 7.2 (Maipo)
> > python 2.7.5
> > celery version 3.1.23
> > kombu  3.0.35
> > rabbitMQ 3.3.5
> >
> > airflow.config is attached
> > logs of scheduler and rabbitmq are too big, i can't attach them here.
> > do you want the end of the log?
> >
> > i'll be happy to provide more info....
> >
> >
> >
> >
> >
> >
> > On Wed, Sep 7, 2016 at 8:40 AM, Bolke de Bruin <bdbruin@gmail.com
> <ma...@gmail.com>> wrote:
> > 1.6.2 is quite old and many updates to the scheduler have been made.
> Please make sure to use 1.7.1.3 or master.
> >
> > Also memory corruption requires more details as that indicates a problem
> with the interpreter itself. Then you would get a core dump and a SIGSEV.
> Did you get those?
> >
> > Bolke
> >
> > Sent from my iPhone
> >
> > > On 7 sep. 2016, at 02:45, Lance Norskog <lance.norskog@gmail.com
> <ma...@gmail.com>> wrote:
> > >
> > > Add your Airflow version and your Python & OS.
> > > I'm on Py 2.7, Airflow 1.6.2 and have seen few different manifestions
> of
> > > memory corruption.
> > >
> > >
> > >> On Sun, Sep 4, 2016 at 1:38 PM, Bolke de Bruin <bdbruin@gmail.com
> <ma...@gmail.com>> wrote:
> > >>
> > >> That would be interesting, but dying - are you sure you are not
> running
> > >> with num_runs enabled?
> > >>
> > >> Yes please specify details.
> > >>
> > >> Verstuurd vanaf mijn iPad
> > >>
> > >> Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <andrewp@apache.org
> <ma...@apache.org>> het
> > >> volgende geschreven:
> > >>
> > >>>> First and foremost I am assuming that getting “stuck” is only
> > >>>> happening when using a CeleryExecutor.
> > >>>
> > >>> We have seen repeated instanced of the scheduler "dying" - i.e. no
> more
> > >> scheduler threads in a ps output - with LocalExecutor too. If you
> feel this
> > >> fits the description of "getting stuck", happy to provide more detail
> to
> > >> try to get to a reproducible situation.
> > >>>
> > >>> Regards
> > >>>
> > >>> ap
> > >
> > >
> > >
> > > --
> > > Lance Norskog
> > > lance.norskog@gmail.com <ma...@gmail.com>
> > > Redwood City, CA
> >
>
>

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
Thanks!

Apache scrubs attachments. Can you please put it somewhere where it can be downloaded (pastebin).

Full logs are more helpful than just the end. If those can be downloaded that would be nice.

Bolke


> Op 7 sep. 2016, om 08:24 heeft הילה ויזן <hi...@gmail.com> het volgende geschreven:
> 
> Hi,
> we face the same issue with latest version.
> environment:
> airflow 1.7.1.3.
> postgress 9.2.13 (backend DB)
> OS   Red Hat Enterprise Linux Server 7.2 (Maipo)
> python 2.7.5
> celery version 3.1.23
> kombu  3.0.35
> rabbitMQ 3.3.5
> 
> airflow.config is attached
> logs of scheduler and rabbitmq are too big, i can't attach them here.
> do you want the end of the log?
> 
> i'll be happy to provide more info....
> 
> 
> 
> 
> 
> 
> On Wed, Sep 7, 2016 at 8:40 AM, Bolke de Bruin <bdbruin@gmail.com <ma...@gmail.com>> wrote:
> 1.6.2 is quite old and many updates to the scheduler have been made. Please make sure to use 1.7.1.3 or master.
> 
> Also memory corruption requires more details as that indicates a problem with the interpreter itself. Then you would get a core dump and a SIGSEV. Did you get those?
> 
> Bolke
> 
> Sent from my iPhone
> 
> > On 7 sep. 2016, at 02:45, Lance Norskog <lance.norskog@gmail.com <ma...@gmail.com>> wrote:
> >
> > Add your Airflow version and your Python & OS.
> > I'm on Py 2.7, Airflow 1.6.2 and have seen few different manifestions of
> > memory corruption.
> >
> >
> >> On Sun, Sep 4, 2016 at 1:38 PM, Bolke de Bruin <bdbruin@gmail.com <ma...@gmail.com>> wrote:
> >>
> >> That would be interesting, but dying - are you sure you are not running
> >> with num_runs enabled?
> >>
> >> Yes please specify details.
> >>
> >> Verstuurd vanaf mijn iPad
> >>
> >> Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <andrewp@apache.org <ma...@apache.org>> het
> >> volgende geschreven:
> >>
> >>>> First and foremost I am assuming that getting “stuck” is only
> >>>> happening when using a CeleryExecutor.
> >>>
> >>> We have seen repeated instanced of the scheduler "dying" - i.e. no more
> >> scheduler threads in a ps output - with LocalExecutor too. If you feel this
> >> fits the description of "getting stuck", happy to provide more detail to
> >> try to get to a reproducible situation.
> >>>
> >>> Regards
> >>>
> >>> ap
> >
> >
> >
> > --
> > Lance Norskog
> > lance.norskog@gmail.com <ma...@gmail.com>
> > Redwood City, CA
> 


Re: Scheduler getting stuck - request for details

Posted by הילה ויזן <hi...@gmail.com>.
Hi,
we face the same issue with latest version.
*environment:*
*airflow* 1.7.1.3.
*postgress *9.2.13 (backend DB)
*OS*   Red Hat Enterprise Linux Server 7.2 (Maipo)
*python* 2.7.5
*celery* version 3.1.23
*kombu  *3.0.35
*rabbitMQ *3.3.5

airflow.config is attached

logs of scheduler and rabbitmq are too big, i can't attach them here.

do you want the end of the log?


i'll be happy to provide more info....







On Wed, Sep 7, 2016 at 8:40 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> 1.6.2 is quite old and many updates to the scheduler have been made.
> Please make sure to use 1.7.1.3 or master.
>
> Also memory corruption requires more details as that indicates a problem
> with the interpreter itself. Then you would get a core dump and a SIGSEV.
> Did you get those?
>
> Bolke
>
> Sent from my iPhone
>
> > On 7 sep. 2016, at 02:45, Lance Norskog <la...@gmail.com> wrote:
> >
> > Add your Airflow version and your Python & OS.
> > I'm on Py 2.7, Airflow 1.6.2 and have seen few different manifestions of
> > memory corruption.
> >
> >
> >> On Sun, Sep 4, 2016 at 1:38 PM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>
> >> That would be interesting, but dying - are you sure you are not running
> >> with num_runs enabled?
> >>
> >> Yes please specify details.
> >>
> >> Verstuurd vanaf mijn iPad
> >>
> >> Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <an...@apache.org> het
> >> volgende geschreven:
> >>
> >>>> First and foremost I am assuming that getting “stuck” is only
> >>>> happening when using a CeleryExecutor.
> >>>
> >>> We have seen repeated instanced of the scheduler "dying" - i.e. no more
> >> scheduler threads in a ps output - with LocalExecutor too. If you feel
> this
> >> fits the description of "getting stuck", happy to provide more detail to
> >> try to get to a reproducible situation.
> >>>
> >>> Regards
> >>>
> >>> ap
> >
> >
> >
> > --
> > Lance Norskog
> > lance.norskog@gmail.com
> > Redwood City, CA
>

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
1.6.2 is quite old and many updates to the scheduler have been made. Please make sure to use 1.7.1.3 or master. 

Also memory corruption requires more details as that indicates a problem with the interpreter itself. Then you would get a core dump and a SIGSEV. Did you get those?

Bolke

Sent from my iPhone

> On 7 sep. 2016, at 02:45, Lance Norskog <la...@gmail.com> wrote:
> 
> Add your Airflow version and your Python & OS.
> I'm on Py 2.7, Airflow 1.6.2 and have seen few different manifestions of
> memory corruption.
> 
> 
>> On Sun, Sep 4, 2016 at 1:38 PM, Bolke de Bruin <bd...@gmail.com> wrote:
>> 
>> That would be interesting, but dying - are you sure you are not running
>> with num_runs enabled?
>> 
>> Yes please specify details.
>> 
>> Verstuurd vanaf mijn iPad
>> 
>> Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <an...@apache.org> het
>> volgende geschreven:
>> 
>>>> First and foremost I am assuming that getting “stuck” is only
>>>> happening when using a CeleryExecutor.
>>> 
>>> We have seen repeated instanced of the scheduler "dying" - i.e. no more
>> scheduler threads in a ps output - with LocalExecutor too. If you feel this
>> fits the description of "getting stuck", happy to provide more detail to
>> try to get to a reproducible situation.
>>> 
>>> Regards
>>> 
>>> ap
> 
> 
> 
> -- 
> Lance Norskog
> lance.norskog@gmail.com
> Redwood City, CA

Re: Scheduler getting stuck - request for details

Posted by Lance Norskog <la...@gmail.com>.
Add your Airflow version and your Python & OS.
I'm on Py 2.7, Airflow 1.6.2 and have seen few different manifestions of
memory corruption.


On Sun, Sep 4, 2016 at 1:38 PM, Bolke de Bruin <bd...@gmail.com> wrote:

> That would be interesting, but dying - are you sure you are not running
> with num_runs enabled?
>
> Yes please specify details.
>
> Verstuurd vanaf mijn iPad
>
> Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <an...@apache.org> het
> volgende geschreven:
>
> >> First and foremost I am assuming that getting “stuck” is only
> >> happening when using a CeleryExecutor.
> >
> > We have seen repeated instanced of the scheduler "dying" - i.e. no more
> scheduler threads in a ps output - with LocalExecutor too. If you feel this
> fits the description of "getting stuck", happy to provide more detail to
> try to get to a reproducible situation.
> >
> > Regards
> >
> > ap
>



-- 
Lance Norskog
lance.norskog@gmail.com
Redwood City, CA

Re: Scheduler getting stuck - request for details

Posted by Bolke de Bruin <bd...@gmail.com>.
That would be interesting, but dying - are you sure you are not running with num_runs enabled? 

Yes please specify details. 

Verstuurd vanaf mijn iPad

Op 4 sep. 2016 om 15:57 heeft Andrew Phillips <an...@apache.org> het volgende geschreven:

>> First and foremost I am assuming that getting “stuck” is only
>> happening when using a CeleryExecutor.
> 
> We have seen repeated instanced of the scheduler "dying" - i.e. no more scheduler threads in a ps output - with LocalExecutor too. If you feel this fits the description of "getting stuck", happy to provide more detail to try to get to a reproducible situation.
> 
> Regards
> 
> ap

Re: Scheduler getting stuck - request for details

Posted by Andrew Phillips <an...@apache.org>.
> First and foremost I am assuming that getting \u201cstuck\u201d is only
> happening when using a CeleryExecutor.

We have seen repeated instanced of the scheduler "dying" - i.e. no more 
scheduler threads in a ps output - with LocalExecutor too. If you feel 
this fits the description of "getting stuck", happy to provide more 
detail to try to get to a reproducible situation.

Regards

ap