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 2017/01/20 09:01:26 UTC

Experiences with 1.8.0

This is to report back on some of the (early) experiences we have with Airflow 1.8.0 (beta 1 at the moment):

1. The UI does not show faulty DAG, leading to confusion for developers. 
When a faulty dag is placed in the dags folder the UI would report a parsing error. Now it doesn’t due to the separate parising (but not reporting back errors)

2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
We run in a secure environment which requires this variable to be whitelisted if it is modified (needs to be added to UPDATING.md)

3. DagRuns do not exist for certain tasks, but don’t get fixed
Log gets flooded without a suggestion what to do

4. At start up all running dag_runs are being checked, we seemed to have a lot of “left over” dag_runs (couple of thousand)
- Checking was logged to INFO -> requires a fsync for every log message making it very slow
- Checking would happen at every restart, but dag_runs’ states were not being updated
- These dag_runs would never er be marked anything else than running for some reason
-> Applied work around to update all dag_run in sql before a certain date to -> finished
-> need to investigate why dag_runs did not get marked “finished/failed” 

5. Our umask is set to 027 


Experiences with 1.8.0 (updated)

Posted by Bolke de Bruin <bd...@gmail.com>.
— continued accidentally pressed send —

This is to report back on some of the (early) experiences we have with Airflow 1.8.0 (beta 1 at the moment):

1. The UI does not show faulty DAG, leading to confusion for developers. 
When a faulty dag is placed in the dags folder the UI would report a parsing error. Now it doesn’t due to the separate parising (but not reporting back errors)

2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
We run in a secure environment which requires this variable to be whitelisted if it is modified (needs to be added to UPDATING.md)

3. DagRuns do not exist for certain tasks, but don’t get fixed
Log gets flooded without a suggestion what to do

4. At start up all running dag_runs are being checked, we seemed to have a lot of “left over” dag_runs (couple of thousand)
- Checking was logged to INFO -> requires a fsync for every log message making it very slow
- Checking would happen at every restart, but dag_runs’ states were not being updated
- These dag_runs would never er be marked anything else than running for some reason
-> Applied work around to update all dag_run in sql before a certain date to -> finished
-> need to investigate why dag_runs did not get marked “finished/failed” 

5. Our umask is set to 027, but scheduler logging directories were created 777
- Cannot reproduce this locally, so we need to investigate.

6. Scanning the DAG dir only every 5 minutes by default seems very slow in more “dev/prod” mixed environments
-> Default should be set lower (30s) with best practice for prod environments set to maybe 300s


That’s it for now. Nothing really a show stopper I guess, but #4 is something we need to take care of. Rest can be fixed with small updates or good documentation.

Will release Beta 2 today, that will contain the major feature of cgroups+impersonation, but will not contain fixes yet for the above.

Bolke



Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Hey all,

I have sent in a PR and JIRA here:

https://github.com/apache/incubator-airflow/pull/2021
https://issues.apache.org/jira/browse/AIRFLOW-807

Please have a look.

EDIT: I see Arthur just did haha.

Cheers,
Chris

On Tue, Jan 24, 2017 at 9:41 PM, Chris Riccomini <cr...@apache.org>
wrote:

> @Max, ran both analyze/optimize. Didn't help. Explain still tries to use
> `state` index. :(
>
> On Tue, Jan 24, 2017 at 5:54 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>
>> I have looked into the issue and it is harmless. What happens is that a
>> TaskInstance writes “success” to the database and the monitoring catches
>> this change before the process is exited. It reports wrongly (ie. queued)
>> as self.task_instance is not updated. I have opened AIRFLOW-798, but will
>> not consider it as a blocker for 1.8.0.
>>
>> - Bolke
>>
>>
>> > On 24 Jan 2017, at 10:38, Bolke de Bruin <bd...@gmail.com> wrote:
>> >
>> > Hey Chris,
>> >
>> > Could you dive into the below a bit more? I don’t like that the
>> LocalTask job is saying the external state is set to queued, although it
>> might just be that the monitoring just does not take into account the
>> queued state which it should (still I am wondering why it happens after the
>> task has finished - maybe db locking interferes). I also see it with my
>> tasks so will dive in myself as well.
>> >
>> > Bolke
>> >
>> >> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>> wrote:
>> >>
>> >> Also, seeing this in EVERY task that runs:
>> >>
>> >> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>> >> instance has been externally set to queued. Taking the poison pill. So
>> >> long.
>> >> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>> return code 0
>> >>
>> >>
>> >> All successful tasks are showing this at the end of their logs. Is this
>> >> normal?
>> >>
>> >> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>> criccomini@apache.org>
>> >> wrote:
>> >>
>> >>> Hey all,
>> >>>
>> >>> I've upgraded on production. Things seem to be working so far (only
>> been
>> >>> an hour), but I am seeing this in the scheduler logs:
>> >>>
>> >>> File Path
>>  PID
>> >>> Runtime    Last Runtime    Last Run
>> >>> ------------------------------------------------------------------
>> -----
>> >>> ---------  --------------  -------------------
>> >>> ...
>> >>> /etc/airflow/dags/dags/elt/el/db.py
>>  24793
>> >>> 43.41s     986.63s         2017-01-23T20:04:09
>> >>> ...
>> >>>
>> >>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>> >>> what's causing this? Scheduler config:
>> >>>
>> >>> [scheduler]
>> >>> job_heartbeat_sec = 5
>> >>> scheduler_heartbeat_sec = 5
>> >>> max_threads = 2
>> >>> child_process_log_directory = /var/log/airflow/scheduler
>> >>>
>> >>> The db.py file, itself, doesn't interact with any outside systems, so
>> I
>> >>> would have expected this to not take so long. It does, however,
>> >>> programmatically generate many DAGs within the single .py file.
>> >>>
>> >>> A snippet of the scheduler log is here:
>> >>>
>> >>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> >>>
>> >>> Note how there are 10-15 second gaps occasionally. Any idea what's
>> going
>> >>> on?
>> >>>
>> >>> Cheers,
>> >>> Chris
>> >>>
>> >>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> >>>
>> >>>> I created:
>> >>>>
>> >>>> - AIRFLOW-791: At start up all running dag_runs are being checked,
>> but
>> >>>> not fixed
>> >>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
>> fixed
>> >>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>> >>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>> interval
>> >>>> was specified
>> >>>>
>> >>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>> >>>>
>> >>>> Please note that I don't consider any of these blockers for a
>> release of
>> >>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
>> on Feb
>> >>>> 2. However if people are using a restarting scheduler (run_duration
>> is set)
>> >>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>> >>>> workaround for this would be nice (we just updated dag_runs directly
>> in the
>> >>>> database to say ‘finished’ before a certain date, but we are also
>> not using
>> >>>> the run_duration).
>> >>>>
>> >>>> Bolke
>> >>>>
>> >>>>
>> >>>>
>> >>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>> >>>>>
>> >>>>> Will do. And thanks.
>> >>>>>
>> >>>>> Adding another issue:
>> >>>>>
>> >>>>> * Some of our DAGs are not getting scheduled for some unknown
>> reason.
>> >>>>> Need to investigate why.
>> >>>>>
>> >>>>> Related but not root cause:
>> >>>>> * Logging is so chatty that it gets really hard to find the real
>> issue
>> >>>>>
>> >>>>> Bolke.
>> >>>>>
>> >>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>> .INVALID>
>> >>>> wrote:
>> >>>>>>
>> >>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>> >>>> others
>> >>>>>> are too. Do you mind creating jiras for these since you have the
>> full
>> >>>>>> context? I have created a JIRA for (1) and have assigned it to
>> myself:
>> >>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>> >>>>>>
>> >>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bdbruin@gmail.com
>> >
>> >>>> wrote:
>> >>>>>>
>> >>>>>>> This is to report back on some of the (early) experiences we have
>> with
>> >>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>> >>>>>>>
>> >>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>> >>>> developers.
>> >>>>>>> When a faulty dag is placed in the dags folder the UI would
>> report a
>> >>>>>>> parsing error. Now it doesn’t due to the separate parising (but
>> not
>> >>>>>>> reporting back errors)
>> >>>>>>>
>> >>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> >>>>>>> We run in a secure environment which requires this variable to be
>> >>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> >>>>>>>
>> >>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> >>>>>>> Log gets flooded without a suggestion what to do
>> >>>>>>>
>> >>>>>>> 4. At start up all running dag_runs are being checked, we seemed
>> to
>> >>>> have a
>> >>>>>>> lot of “left over” dag_runs (couple of thousand)
>> >>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>> >>>> message
>> >>>>>>> making it very slow
>> >>>>>>> - Checking would happen at every restart, but dag_runs’ states
>> were
>> >>>> not
>> >>>>>>> being updated
>> >>>>>>> - These dag_runs would never er be marked anything else than
>> running
>> >>>> for
>> >>>>>>> some reason
>> >>>>>>> -> Applied work around to update all dag_run in sql before a
>> certain
>> >>>> date
>> >>>>>>> to -> finished
>> >>>>>>> -> need to investigate why dag_runs did not get marked
>> >>>> “finished/failed”
>> >>>>>>>
>> >>>>>>> 5. Our umask is set to 027
>> >>>>>>>
>> >>>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>
>> >
>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
@Max, ran both analyze/optimize. Didn't help. Explain still tries to use
`state` index. :(

On Tue, Jan 24, 2017 at 5:54 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> I have looked into the issue and it is harmless. What happens is that a
> TaskInstance writes “success” to the database and the monitoring catches
> this change before the process is exited. It reports wrongly (ie. queued)
> as self.task_instance is not updated. I have opened AIRFLOW-798, but will
> not consider it as a blocker for 1.8.0.
>
> - Bolke
>
>
> > On 24 Jan 2017, at 10:38, Bolke de Bruin <bd...@gmail.com> wrote:
> >
> > Hey Chris,
> >
> > Could you dive into the below a bit more? I don’t like that the
> LocalTask job is saying the external state is set to queued, although it
> might just be that the monitoring just does not take into account the
> queued state which it should (still I am wondering why it happens after the
> task has finished - maybe db locking interferes). I also see it with my
> tasks so will dive in myself as well.
> >
> > Bolke
> >
> >> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
> wrote:
> >>
> >> Also, seeing this in EVERY task that runs:
> >>
> >> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
> >> instance has been externally set to queued. Taking the poison pill. So
> >> long.
> >> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return
> code 0
> >>
> >>
> >> All successful tasks are showing this at the end of their logs. Is this
> >> normal?
> >>
> >> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
> criccomini@apache.org>
> >> wrote:
> >>
> >>> Hey all,
> >>>
> >>> I've upgraded on production. Things seem to be working so far (only
> been
> >>> an hour), but I am seeing this in the scheduler logs:
> >>>
> >>> File Path
>  PID
> >>> Runtime    Last Runtime    Last Run
> >>> ------------------------------------------------------------------
> -----
> >>> ---------  --------------  -------------------
> >>> ...
> >>> /etc/airflow/dags/dags/elt/el/db.py
>  24793
> >>> 43.41s     986.63s         2017-01-23T20:04:09
> >>> ...
> >>>
> >>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
> >>> what's causing this? Scheduler config:
> >>>
> >>> [scheduler]
> >>> job_heartbeat_sec = 5
> >>> scheduler_heartbeat_sec = 5
> >>> max_threads = 2
> >>> child_process_log_directory = /var/log/airflow/scheduler
> >>>
> >>> The db.py file, itself, doesn't interact with any outside systems, so I
> >>> would have expected this to not take so long. It does, however,
> >>> programmatically generate many DAGs within the single .py file.
> >>>
> >>> A snippet of the scheduler log is here:
> >>>
> >>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
> >>>
> >>> Note how there are 10-15 second gaps occasionally. Any idea what's
> going
> >>> on?
> >>>
> >>> Cheers,
> >>> Chris
> >>>
> >>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>>
> >>>> I created:
> >>>>
> >>>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
> >>>> not fixed
> >>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
> fixed
> >>>> - AIRFLOW-788: Context unexpectedly added to hive conf
> >>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
> interval
> >>>> was specified
> >>>>
> >>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
> >>>>
> >>>> Please note that I don't consider any of these blockers for a release
> of
> >>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
> on Feb
> >>>> 2. However if people are using a restarting scheduler (run_duration
> is set)
> >>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> >>>> workaround for this would be nice (we just updated dag_runs directly
> in the
> >>>> database to say ‘finished’ before a certain date, but we are also not
> using
> >>>> the run_duration).
> >>>>
> >>>> Bolke
> >>>>
> >>>>
> >>>>
> >>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> >>>>>
> >>>>> Will do. And thanks.
> >>>>>
> >>>>> Adding another issue:
> >>>>>
> >>>>> * Some of our DAGs are not getting scheduled for some unknown reason.
> >>>>> Need to investigate why.
> >>>>>
> >>>>> Related but not root cause:
> >>>>> * Logging is so chatty that it gets really hard to find the real
> issue
> >>>>>
> >>>>> Bolke.
> >>>>>
> >>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com.
> INVALID>
> >>>> wrote:
> >>>>>>
> >>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
> >>>> others
> >>>>>> are too. Do you mind creating jiras for these since you have the
> full
> >>>>>> context? I have created a JIRA for (1) and have assigned it to
> myself:
> >>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
> >>>>>>
> >>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>>> This is to report back on some of the (early) experiences we have
> with
> >>>>>>> Airflow 1.8.0 (beta 1 at the moment):
> >>>>>>>
> >>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
> >>>> developers.
> >>>>>>> When a faulty dag is placed in the dags folder the UI would report
> a
> >>>>>>> parsing error. Now it doesn’t due to the separate parising (but not
> >>>>>>> reporting back errors)
> >>>>>>>
> >>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >>>>>>> We run in a secure environment which requires this variable to be
> >>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
> >>>>>>>
> >>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> >>>>>>> Log gets flooded without a suggestion what to do
> >>>>>>>
> >>>>>>> 4. At start up all running dag_runs are being checked, we seemed to
> >>>> have a
> >>>>>>> lot of “left over” dag_runs (couple of thousand)
> >>>>>>> - Checking was logged to INFO -> requires a fsync for every log
> >>>> message
> >>>>>>> making it very slow
> >>>>>>> - Checking would happen at every restart, but dag_runs’ states were
> >>>> not
> >>>>>>> being updated
> >>>>>>> - These dag_runs would never er be marked anything else than
> running
> >>>> for
> >>>>>>> some reason
> >>>>>>> -> Applied work around to update all dag_run in sql before a
> certain
> >>>> date
> >>>>>>> to -> finished
> >>>>>>> -> need to investigate why dag_runs did not get marked
> >>>> “finished/failed”
> >>>>>>>
> >>>>>>> 5. Our umask is set to 027
> >>>>>>>
> >>>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >
>
>

Re: Experiences with 1.8.0

Posted by Bolke de Bruin <bd...@gmail.com>.
I have looked into the issue and it is harmless. What happens is that a TaskInstance writes “success” to the database and the monitoring catches this change before the process is exited. It reports wrongly (ie. queued) as self.task_instance is not updated. I have opened AIRFLOW-798, but will not consider it as a blocker for 1.8.0.

- Bolke


> On 24 Jan 2017, at 10:38, Bolke de Bruin <bd...@gmail.com> wrote:
> 
> Hey Chris,
> 
> Could you dive into the below a bit more? I don’t like that the LocalTask job is saying the external state is set to queued, although it might just be that the monitoring just does not take into account the queued state which it should (still I am wondering why it happens after the task has finished - maybe db locking interferes). I also see it with my tasks so will dive in myself as well.
> 
> Bolke
> 
>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org> wrote:
>> 
>> Also, seeing this in EVERY task that runs:
>> 
>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>> instance has been externally set to queued. Taking the poison pill. So
>> long.
>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return code 0
>> 
>> 
>> All successful tasks are showing this at the end of their logs. Is this
>> normal?
>> 
>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
>> wrote:
>> 
>>> Hey all,
>>> 
>>> I've upgraded on production. Things seem to be working so far (only been
>>> an hour), but I am seeing this in the scheduler logs:
>>> 
>>> File Path                                                             PID
>>> Runtime    Last Runtime    Last Run
>>> ------------------------------------------------------------------  -----
>>> ---------  --------------  -------------------
>>> ...
>>> /etc/airflow/dags/dags/elt/el/db.py                                 24793
>>> 43.41s     986.63s         2017-01-23T20:04:09
>>> ...
>>> 
>>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>>> what's causing this? Scheduler config:
>>> 
>>> [scheduler]
>>> job_heartbeat_sec = 5
>>> scheduler_heartbeat_sec = 5
>>> max_threads = 2
>>> child_process_log_directory = /var/log/airflow/scheduler
>>> 
>>> The db.py file, itself, doesn't interact with any outside systems, so I
>>> would have expected this to not take so long. It does, however,
>>> programmatically generate many DAGs within the single .py file.
>>> 
>>> A snippet of the scheduler log is here:
>>> 
>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>>> 
>>> Note how there are 10-15 second gaps occasionally. Any idea what's going
>>> on?
>>> 
>>> Cheers,
>>> Chris
>>> 
>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>>> 
>>>> I created:
>>>> 
>>>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>>>> not fixed
>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
>>>> was specified
>>>> 
>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>>>> 
>>>> Please note that I don't consider any of these blockers for a release of
>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb
>>>> 2. However if people are using a restarting scheduler (run_duration is set)
>>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>>>> workaround for this would be nice (we just updated dag_runs directly in the
>>>> database to say ‘finished’ before a certain date, but we are also not using
>>>> the run_duration).
>>>> 
>>>> Bolke
>>>> 
>>>> 
>>>> 
>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>> 
>>>>> Will do. And thanks.
>>>>> 
>>>>> Adding another issue:
>>>>> 
>>>>> * Some of our DAGs are not getting scheduled for some unknown reason.
>>>>> Need to investigate why.
>>>>> 
>>>>> Related but not root cause:
>>>>> * Logging is so chatty that it gets really hard to find the real issue
>>>>> 
>>>>> Bolke.
>>>>> 
>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
>>>> wrote:
>>>>>> 
>>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>>>> others
>>>>>> are too. Do you mind creating jiras for these since you have the full
>>>>>> context? I have created a JIRA for (1) and have assigned it to myself:
>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>>> 
>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>>>> wrote:
>>>>>> 
>>>>>>> This is to report back on some of the (early) experiences we have with
>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>>>> 
>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>>> developers.
>>>>>>> When a faulty dag is placed in the dags folder the UI would report a
>>>>>>> parsing error. Now it doesn’t due to the separate parising (but not
>>>>>>> reporting back errors)
>>>>>>> 
>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>>>> We run in a secure environment which requires this variable to be
>>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>>>>>>> 
>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>>>>>> Log gets flooded without a suggestion what to do
>>>>>>> 
>>>>>>> 4. At start up all running dag_runs are being checked, we seemed to
>>>> have a
>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>>>> message
>>>>>>> making it very slow
>>>>>>> - Checking would happen at every restart, but dag_runs’ states were
>>>> not
>>>>>>> being updated
>>>>>>> - These dag_runs would never er be marked anything else than running
>>>> for
>>>>>>> some reason
>>>>>>> -> Applied work around to update all dag_run in sql before a certain
>>>> date
>>>>>>> to -> finished
>>>>>>> -> need to investigate why dag_runs did not get marked
>>>> “finished/failed”
>>>>>>> 
>>>>>>> 5. Our umask is set to 027
>>>>>>> 
>>>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
> 


Re: Experiences with 1.8.0

Posted by Bolke de Bruin <bd...@gmail.com>.
Hey Chris,

Could you dive into the below a bit more? I don’t like that the LocalTask job is saying the external state is set to queued, although it might just be that the monitoring just does not take into account the queued state which it should (still I am wondering why it happens after the task has finished - maybe db locking interferes). I also see it with my tasks so will dive in myself as well.

Bolke

> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org> wrote:
> 
> Also, seeing this in EVERY task that runs:
> 
> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
> instance has been externally set to queued. Taking the poison pill. So
> long.
> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return code 0
> 
> 
> All successful tasks are showing this at the end of their logs. Is this
> normal?
> 
> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
> wrote:
> 
>> Hey all,
>> 
>> I've upgraded on production. Things seem to be working so far (only been
>> an hour), but I am seeing this in the scheduler logs:
>> 
>> File Path                                                             PID
>> Runtime    Last Runtime    Last Run
>> ------------------------------------------------------------------  -----
>> ---------  --------------  -------------------
>> ...
>> /etc/airflow/dags/dags/elt/el/db.py                                 24793
>> 43.41s     986.63s         2017-01-23T20:04:09
>> ...
>> 
>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>> what's causing this? Scheduler config:
>> 
>> [scheduler]
>> job_heartbeat_sec = 5
>> scheduler_heartbeat_sec = 5
>> max_threads = 2
>> child_process_log_directory = /var/log/airflow/scheduler
>> 
>> The db.py file, itself, doesn't interact with any outside systems, so I
>> would have expected this to not take so long. It does, however,
>> programmatically generate many DAGs within the single .py file.
>> 
>> A snippet of the scheduler log is here:
>> 
>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> 
>> Note how there are 10-15 second gaps occasionally. Any idea what's going
>> on?
>> 
>> Cheers,
>> Chris
>> 
>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>> 
>>> I created:
>>> 
>>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>>> not fixed
>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
>>> was specified
>>> 
>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>>> 
>>> Please note that I don't consider any of these blockers for a release of
>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb
>>> 2. However if people are using a restarting scheduler (run_duration is set)
>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>>> workaround for this would be nice (we just updated dag_runs directly in the
>>> database to say ‘finished’ before a certain date, but we are also not using
>>> the run_duration).
>>> 
>>> Bolke
>>> 
>>> 
>>> 
>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>>>> 
>>>> Will do. And thanks.
>>>> 
>>>> Adding another issue:
>>>> 
>>>> * Some of our DAGs are not getting scheduled for some unknown reason.
>>>> Need to investigate why.
>>>> 
>>>> Related but not root cause:
>>>> * Logging is so chatty that it gets really hard to find the real issue
>>>> 
>>>> Bolke.
>>>> 
>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
>>> wrote:
>>>>> 
>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>>> others
>>>>> are too. Do you mind creating jiras for these since you have the full
>>>>> context? I have created a JIRA for (1) and have assigned it to myself:
>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>> 
>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>>> wrote:
>>>>> 
>>>>>> This is to report back on some of the (early) experiences we have with
>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>>> 
>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>> developers.
>>>>>> When a faulty dag is placed in the dags folder the UI would report a
>>>>>> parsing error. Now it doesn’t due to the separate parising (but not
>>>>>> reporting back errors)
>>>>>> 
>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>>> We run in a secure environment which requires this variable to be
>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>>>>>> 
>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>>>>> Log gets flooded without a suggestion what to do
>>>>>> 
>>>>>> 4. At start up all running dag_runs are being checked, we seemed to
>>> have a
>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>>> message
>>>>>> making it very slow
>>>>>> - Checking would happen at every restart, but dag_runs’ states were
>>> not
>>>>>> being updated
>>>>>> - These dag_runs would never er be marked anything else than running
>>> for
>>>>>> some reason
>>>>>> -> Applied work around to update all dag_run in sql before a certain
>>> date
>>>>>> to -> finished
>>>>>> -> need to investigate why dag_runs did not get marked
>>> “finished/failed”
>>>>>> 
>>>>>> 5. Our umask is set to 027
>>>>>> 
>>>>>> 
>>>> 
>>> 
>>> 
>> 


Re: Experiences with 1.8.0

Posted by Maxime Beauchemin <ma...@gmail.com>.
Can you rebuild your indexes and recompute the table's stats and see if the
optimizer is still off tracks?

Assuming InnoDB and from memory:

OPTIMIZE TABLE task_instances;
ANALYZE TABLE task_instances;

Max

On Mon, Jan 23, 2017 at 3:45 PM, Arthur Wiedmer <ar...@gmail.com>
wrote:

> Maybe we can start with
> " .with_hint(TI, 'USE INDEX (PRIMARY)', dialect_name='mysql')"
>
> and see if other databases exhibit the same query plan issue ?
>
> Best,
> Arthur
>
> On Mon, Jan 23, 2017 at 3:27 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
> > With this patch:
> >
> > $ git diff
> > diff --git a/airflow/jobs.py b/airflow/jobs.py
> > index f1de333..9d08e75 100644
> > --- a/airflow/jobs.py
> > +++ b/airflow/jobs.py
> > @@ -544,6 +544,7 @@ class SchedulerJob(BaseJob):
> >              .query(
> >                  TI.task_id,
> >                  func.max(TI.execution_date).label('max_ti'))
> > +            .with_hint(TI, 'USE INDEX (PRIMARY)')
> >              .filter(TI.dag_id == dag.dag_id)
> >              .filter(TI.state == State.SUCCESS)
> >              .filter(TI.task_id.in_(dag.task_ids))
> >
> > The db.py file parses in 90s. About 10x faster.
> >
> > Th slow_query log is not showing any offending queries anymore. I will
> open
> > a JIRA for this. Note that with_hint stuff I put in is MySQL specific, I
> > think. Any one have any other ideas on how to affect the equivalent
> > outcome?
> >
> > On Mon, Jan 23, 2017 at 3:08 PM, Chris Riccomini <cr...@apache.org>
> > wrote:
> >
> > > OK, it's using `state` instead of PRIMARY. Using PRIMARY with a hint,
> > > query takes .47s. Without hint, 10s. Going to try and patch.
> > >
> > > On Mon, Jan 23, 2017 at 2:57 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> This inner query takes 10s:
> > >>
> > >> SELECT task_instance.task_id AS task_id, max(task_instance.execution_
> > date)
> > >> AS max_ti
> > >> FROM task_instance
> > >> WHERE task_instance.dag_id = 'dag1' AND task_instance.state =
> 'success'
> > >> AND task_instance.task_id IN ('t1', 't2') GROUP BY
> task_instance.task_id
> > >>
> > >> Explain seems OK:
> > >>
> > >> +----+-------------+---------------+------+-----------------
> > >> -----------------------------------+----------+---------+---
> > >> ----+-------+--------------------------+
> > >> | id | select_type | table         | type | possible_keys
> > >>                      | key      | key_len | ref   | rows  | Extra
> > >>          |
> > >> +----+-------------+---------------+------+-----------------
> > >> -----------------------------------+----------+---------+---
> > >> ----+-------+--------------------------+
> > >> |  1 | SIMPLE      | task_instance | ref  |
> > PRIMARY,ti_dag_state,ti_pool,ti_state_lkp,ti_state
> > >> | ti_state | 63      | const | 81898 | Using where; Using index |
> > >> +----+-------------+---------------+------+-----------------
> > >> -----------------------------------+----------+---------+---
> > >> ----+-------+--------------------------+
> > >>
> > >> A look at the query indicates that it's using the state field as its
> > >> index lookup, which isn't good.
> > >>
> > >> On Mon, Jan 23, 2017 at 2:49 PM, Chris Riccomini <
> criccomini@apache.org
> > >
> > >> wrote:
> > >>
> > >>> It's this query:
> > >>>
> > >>> SELECT task_instance.task_id AS task_instance_task_id,
> > >>> task_instance.dag_id AS task_instance_dag_id,
> > task_instance.execution_date
> > >>> AS task_instance_execution_date, task_instance.start_date AS
> > >>> task_instance_start_date, task_instance.end_date AS
> > task_instance_end_date,
> > >>> task_instance.duration AS task_instance_duration, task_instance.state
> > AS
> > >>> task_instance_state, task_instance.try_number AS
> > task_instance_try_number,
> > >>> task_instance.hostname AS task_instance_hostname,
> > task_instance.unixname AS
> > >>> task_instance_unixname, task_instance.job_id AS task_instance_job_id,
> > >>> task_instance.pool AS task_instance_pool, task_instance.queue AS
> > >>> task_instance_queue, task_instance.priority_weight AS
> > >>> task_instance_priority_weight, task_instance.operator AS
> > >>> task_instance_operator, task_instance.queued_dttm AS
> > >>> task_instance_queued_dttm, task_instance.pid AS task_instance_pid
> > >>> FROM task_instance, (SELECT task_instance.task_id AS task_id,
> > >>> max(task_instance.execution_date) AS max_ti
> > >>> FROM task_instance
> > >>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
> > >>> 'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5',
> > 't6')
> > >>> GROUP BY task_instance.task_id) AS sq
> > >>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id
> =
> > >>> sq.task_id AND task_instance.execution_date = sq.max_ti
> > >>>
> > >>> Taking 10-20s
> > >>>
> > >>> On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <
> > criccomini@apache.org>
> > >>> wrote:
> > >>>
> > >>>> Can confirm it's a slow query on task_instance table. Still digging.
> > >>>> Unfortunately, the query is truncated in my UI right now:
> > >>>>
> > >>>> SELECT task_instance.task_id AS task_instance_...
> > >>>>
> > >>>> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <
> > criccomini@apache.org
> > >>>> > wrote:
> > >>>>
> > >>>>> Digging. Might be a bit.
> > >>>>>
> > >>>>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bdbruin@gmail.com
> >
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Slow query log? Db load?
> > >>>>>>
> > >>>>>> B.
> > >>>>>>
> > >>>>>> Verstuurd vanaf mijn iPad
> > >>>>>>
> > >>>>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <
> > >>>>>> criccomini@apache.org> het volgende geschreven:
> > >>>>>> >
> > >>>>>> > Note: 6.5 million TIs in the task_instance table.
> > >>>>>> >
> > >>>>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
> > >>>>>> criccomini@apache.org>
> > >>>>>> > wrote:
> > >>>>>> >
> > >>>>>> >> Hey Bolke,
> > >>>>>> >>
> > >>>>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is
> almost
> > >>>>>> all free
> > >>>>>> >> as well.
> > >>>>>> >>
> > >>>>>> >> I am thinking that this is DB related, given that it's pausing
> > when
> > >>>>>> >> executing an update. Was looking at the update_state method in
> > >>>>>> models.py,
> > >>>>>> >> which logs right before the 15s pause.
> > >>>>>> >>
> > >>>>>> >> Cheers,
> > >>>>>> >> Chris
> > >>>>>> >>
> > >>>>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <
> > >>>>>> bdbruin@gmail.com>
> > >>>>>> >> wrote:
> > >>>>>> >>
> > >>>>>> >>> Hi Chris,
> > >>>>>> >>>
> > >>>>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that
> the
> > >>>>>> executor
> > >>>>>> >>> is faster in checking the state than the TaskInstance is able
> to
> > >>>>>> exit
> > >>>>>> >>> itself. No, I don’t consider it normal, but it was sometimes
> > >>>>>> reported (
> > >>>>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though
> > >>>>>> not really
> > >>>>>> >>> replicable as of yet.
> > >>>>>> >>>
> > >>>>>> >>> Parsing seems exceptionally slow, it might be worth looking at
> > it
> > >>>>>> with a
> > >>>>>> >>> debugger. Very faint guess might be that something with the
> > >>>>>> multiprocessing
> > >>>>>> >>> part could do something with memory that is costly, but then
> we
> > >>>>>> need to
> > >>>>>> >>> know more about what is running on the system. Never clue of
> > >>>>>> system metrics
> > >>>>>> >>> could be helpful here.
> > >>>>>> >>>
> > >>>>>> >>> Bolke
> > >>>>>> >>>
> > >>>>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <
> > criccomini@apache.org
> > >>>>>> >
> > >>>>>> >>> wrote:
> > >>>>>> >>>>
> > >>>>>> >>>> Also, seeing this in EVERY task that runs:
> > >>>>>> >>>>
> > >>>>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of
> > this
> > >>>>>> >>>> instance has been externally set to queued. Taking the poison
> > >>>>>> pill. So
> > >>>>>> >>>> long.
> > >>>>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited
> > with
> > >>>>>> return
> > >>>>>> >>> code 0
> > >>>>>> >>>>
> > >>>>>> >>>>
> > >>>>>> >>>> All successful tasks are showing this at the end of their
> logs.
> > >>>>>> Is this
> > >>>>>> >>>> normal?
> > >>>>>> >>>>
> > >>>>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
> > >>>>>> >>> criccomini@apache.org>
> > >>>>>> >>>> wrote:
> > >>>>>> >>>>
> > >>>>>> >>>>> Hey all,
> > >>>>>> >>>>>
> > >>>>>> >>>>> I've upgraded on production. Things seem to be working so
> far
> > >>>>>> (only
> > >>>>>> >>> been
> > >>>>>> >>>>> an hour), but I am seeing this in the scheduler logs:
> > >>>>>> >>>>>
> > >>>>>> >>>>> File Path
> > >>>>>> >>> PID
> > >>>>>> >>>>> Runtime    Last Runtime    Last Run
> > >>>>>> >>>>> ------------------------------
> ------------------------------
> > >>>>>> ------
> > >>>>>> >>> -----
> > >>>>>> >>>>> ---------  --------------  -------------------
> > >>>>>> >>>>> ...
> > >>>>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
> > >>>>>> >>> 24793
> > >>>>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
> > >>>>>> >>>>> ...
> > >>>>>> >>>>>
> > >>>>>> >>>>> It seems to be taking more than 15 minutes to parse this
> DAG.
> > >>>>>> Any idea
> > >>>>>> >>>>> what's causing this? Scheduler config:
> > >>>>>> >>>>>
> > >>>>>> >>>>> [scheduler]
> > >>>>>> >>>>> job_heartbeat_sec = 5
> > >>>>>> >>>>> scheduler_heartbeat_sec = 5
> > >>>>>> >>>>> max_threads = 2
> > >>>>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
> > >>>>>> >>>>>
> > >>>>>> >>>>> The db.py file, itself, doesn't interact with any outside
> > >>>>>> systems, so I
> > >>>>>> >>>>> would have expected this to not take so long. It does,
> > however,
> > >>>>>> >>>>> programmatically generate many DAGs within the single .py
> > file.
> > >>>>>> >>>>>
> > >>>>>> >>>>> A snippet of the scheduler log is here:
> > >>>>>> >>>>>
> > >>>>>> >>>>> https://gist.github.com/criccomini/
> a2b2762763c8ba65fefcdd669
> > >>>>>> e8ffd65
> > >>>>>> >>>>>
> > >>>>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea
> > >>>>>> what's
> > >>>>>> >>> going
> > >>>>>> >>>>> on?
> > >>>>>> >>>>>
> > >>>>>> >>>>> Cheers,
> > >>>>>> >>>>> Chris
> > >>>>>> >>>>>
> > >>>>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <
> > >>>>>> bdbruin@gmail.com>
> > >>>>>> >>> wrote:
> > >>>>>> >>>>>
> > >>>>>> >>>>>> I created:
> > >>>>>> >>>>>>
> > >>>>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
> > >>>>>> checked, but
> > >>>>>> >>>>>> not fixed
> > >>>>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but
> > >>>>>> don’t get
> > >>>>>> >>> fixed
> > >>>>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
> > >>>>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong
> > start_date /
> > >>>>>> >>> interval
> > >>>>>> >>>>>> was specified
> > >>>>>> >>>>>>
> > >>>>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out
> > as
> > >>>>>> a PR.
> > >>>>>> >>>>>>
> > >>>>>> >>>>>> Please note that I don't consider any of these blockers
> for a
> > >>>>>> release
> > >>>>>> >>> of
> > >>>>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track
> > for
> > >>>>>> an RC
> > >>>>>> >>> on Feb
> > >>>>>> >>>>>> 2. However if people are using a restarting scheduler
> > >>>>>> (run_duration
> > >>>>>> >>> is set)
> > >>>>>> >>>>>> and have a lot of running dag runs they won’t like
> > >>>>>> AIRFLOW-791. So a
> > >>>>>> >>>>>> workaround for this would be nice (we just updated dag_runs
> > >>>>>> directly
> > >>>>>> >>> in the
> > >>>>>> >>>>>> database to say ‘finished’ before a certain date, but we
> are
> > >>>>>> also not
> > >>>>>> >>> using
> > >>>>>> >>>>>> the run_duration).
> > >>>>>> >>>>>>
> > >>>>>> >>>>>> Bolke
> > >>>>>> >>>>>>
> > >>>>>> >>>>>>
> > >>>>>> >>>>>>
> > >>>>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <
> bdbruin@gmail.com
> > >
> > >>>>>> wrote:
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>> Will do. And thanks.
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>> Adding another issue:
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>> * Some of our DAGs are not getting scheduled for some
> > unknown
> > >>>>>> reason.
> > >>>>>> >>>>>>> Need to investigate why.
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>> Related but not root cause:
> > >>>>>> >>>>>>> * Logging is so chatty that it gets really hard to find
> the
> > >>>>>> real
> > >>>>>> >>> issue
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>> Bolke.
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <
> > >>>>>> dan.davydov@airbnb.com
> > >>>>>> >>> .INVALID>
> > >>>>>> >>>>>> wrote:
> > >>>>>> >>>>>>>>
> > >>>>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and
> > >>>>>> hopefully some
> > >>>>>> >>>>>> others
> > >>>>>> >>>>>>>> are too. Do you mind creating jiras for these since you
> > have
> > >>>>>> the
> > >>>>>> >>> full
> > >>>>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned
> it
> > >>>>>> to
> > >>>>>> >>> myself:
> > >>>>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
> > >>>>>> >>>>>>>>
> > >>>>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
> > >>>>>> bdbruin@gmail.com>
> > >>>>>> >>>>>> wrote:
> > >>>>>> >>>>>>>>
> > >>>>>> >>>>>>>>> This is to report back on some of the (early)
> experiences
> > >>>>>> we have
> > >>>>>> >>> with
> > >>>>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion
> > for
> > >>>>>> >>>>>> developers.
> > >>>>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI
> > would
> > >>>>>> report
> > >>>>>> >>> a
> > >>>>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate
> parising
> > >>>>>> (but not
> > >>>>>> >>>>>>>>> reporting back errors)
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> > >>>>>> >>>>>>>>> We run in a secure environment which requires this
> > variable
> > >>>>>> to be
> > >>>>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
> > >>>>>> UPDATING.md)
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get
> > >>>>>> fixed
> > >>>>>> >>>>>>>>> Log gets flooded without a suggestion what to do
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>> 4. At start up all running dag_runs are being checked,
> we
> > >>>>>> seemed to
> > >>>>>> >>>>>> have a
> > >>>>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
> > >>>>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for
> > every
> > >>>>>> log
> > >>>>>> >>>>>> message
> > >>>>>> >>>>>>>>> making it very slow
> > >>>>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’
> > >>>>>> states were
> > >>>>>> >>>>>> not
> > >>>>>> >>>>>>>>> being updated
> > >>>>>> >>>>>>>>> - These dag_runs would never er be marked anything else
> > than
> > >>>>>> >>> running
> > >>>>>> >>>>>> for
> > >>>>>> >>>>>>>>> some reason
> > >>>>>> >>>>>>>>> -> Applied work around to update all dag_run in sql
> > before a
> > >>>>>> >>> certain
> > >>>>>> >>>>>> date
> > >>>>>> >>>>>>>>> to -> finished
> > >>>>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
> > >>>>>> >>>>>> “finished/failed”
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>> 5. Our umask is set to 027
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>>>
> > >>>>>> >>>>>>>
> > >>>>>> >>>>>>
> > >>>>>> >>>>>>
> > >>>>>> >>>>>
> > >>>>>> >>>
> > >>>>>> >>>
> > >>>>>> >>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >
>

Re: Experiences with 1.8.0

Posted by Arthur Wiedmer <ar...@gmail.com>.
Maybe we can start with
" .with_hint(TI, 'USE INDEX (PRIMARY)', dialect_name='mysql')"

and see if other databases exhibit the same query plan issue ?

Best,
Arthur

On Mon, Jan 23, 2017 at 3:27 PM, Chris Riccomini <cr...@apache.org>
wrote:

> With this patch:
>
> $ git diff
> diff --git a/airflow/jobs.py b/airflow/jobs.py
> index f1de333..9d08e75 100644
> --- a/airflow/jobs.py
> +++ b/airflow/jobs.py
> @@ -544,6 +544,7 @@ class SchedulerJob(BaseJob):
>              .query(
>                  TI.task_id,
>                  func.max(TI.execution_date).label('max_ti'))
> +            .with_hint(TI, 'USE INDEX (PRIMARY)')
>              .filter(TI.dag_id == dag.dag_id)
>              .filter(TI.state == State.SUCCESS)
>              .filter(TI.task_id.in_(dag.task_ids))
>
> The db.py file parses in 90s. About 10x faster.
>
> Th slow_query log is not showing any offending queries anymore. I will open
> a JIRA for this. Note that with_hint stuff I put in is MySQL specific, I
> think. Any one have any other ideas on how to affect the equivalent
> outcome?
>
> On Mon, Jan 23, 2017 at 3:08 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
> > OK, it's using `state` instead of PRIMARY. Using PRIMARY with a hint,
> > query takes .47s. Without hint, 10s. Going to try and patch.
> >
> > On Mon, Jan 23, 2017 at 2:57 PM, Chris Riccomini <cr...@apache.org>
> > wrote:
> >
> >> This inner query takes 10s:
> >>
> >> SELECT task_instance.task_id AS task_id, max(task_instance.execution_
> date)
> >> AS max_ti
> >> FROM task_instance
> >> WHERE task_instance.dag_id = 'dag1' AND task_instance.state = 'success'
> >> AND task_instance.task_id IN ('t1', 't2') GROUP BY task_instance.task_id
> >>
> >> Explain seems OK:
> >>
> >> +----+-------------+---------------+------+-----------------
> >> -----------------------------------+----------+---------+---
> >> ----+-------+--------------------------+
> >> | id | select_type | table         | type | possible_keys
> >>                      | key      | key_len | ref   | rows  | Extra
> >>          |
> >> +----+-------------+---------------+------+-----------------
> >> -----------------------------------+----------+---------+---
> >> ----+-------+--------------------------+
> >> |  1 | SIMPLE      | task_instance | ref  |
> PRIMARY,ti_dag_state,ti_pool,ti_state_lkp,ti_state
> >> | ti_state | 63      | const | 81898 | Using where; Using index |
> >> +----+-------------+---------------+------+-----------------
> >> -----------------------------------+----------+---------+---
> >> ----+-------+--------------------------+
> >>
> >> A look at the query indicates that it's using the state field as its
> >> index lookup, which isn't good.
> >>
> >> On Mon, Jan 23, 2017 at 2:49 PM, Chris Riccomini <criccomini@apache.org
> >
> >> wrote:
> >>
> >>> It's this query:
> >>>
> >>> SELECT task_instance.task_id AS task_instance_task_id,
> >>> task_instance.dag_id AS task_instance_dag_id,
> task_instance.execution_date
> >>> AS task_instance_execution_date, task_instance.start_date AS
> >>> task_instance_start_date, task_instance.end_date AS
> task_instance_end_date,
> >>> task_instance.duration AS task_instance_duration, task_instance.state
> AS
> >>> task_instance_state, task_instance.try_number AS
> task_instance_try_number,
> >>> task_instance.hostname AS task_instance_hostname,
> task_instance.unixname AS
> >>> task_instance_unixname, task_instance.job_id AS task_instance_job_id,
> >>> task_instance.pool AS task_instance_pool, task_instance.queue AS
> >>> task_instance_queue, task_instance.priority_weight AS
> >>> task_instance_priority_weight, task_instance.operator AS
> >>> task_instance_operator, task_instance.queued_dttm AS
> >>> task_instance_queued_dttm, task_instance.pid AS task_instance_pid
> >>> FROM task_instance, (SELECT task_instance.task_id AS task_id,
> >>> max(task_instance.execution_date) AS max_ti
> >>> FROM task_instance
> >>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
> >>> 'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5',
> 't6')
> >>> GROUP BY task_instance.task_id) AS sq
> >>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id =
> >>> sq.task_id AND task_instance.execution_date = sq.max_ti
> >>>
> >>> Taking 10-20s
> >>>
> >>> On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <
> criccomini@apache.org>
> >>> wrote:
> >>>
> >>>> Can confirm it's a slow query on task_instance table. Still digging.
> >>>> Unfortunately, the query is truncated in my UI right now:
> >>>>
> >>>> SELECT task_instance.task_id AS task_instance_...
> >>>>
> >>>> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <
> criccomini@apache.org
> >>>> > wrote:
> >>>>
> >>>>> Digging. Might be a bit.
> >>>>>
> >>>>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Slow query log? Db load?
> >>>>>>
> >>>>>> B.
> >>>>>>
> >>>>>> Verstuurd vanaf mijn iPad
> >>>>>>
> >>>>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <
> >>>>>> criccomini@apache.org> het volgende geschreven:
> >>>>>> >
> >>>>>> > Note: 6.5 million TIs in the task_instance table.
> >>>>>> >
> >>>>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
> >>>>>> criccomini@apache.org>
> >>>>>> > wrote:
> >>>>>> >
> >>>>>> >> Hey Bolke,
> >>>>>> >>
> >>>>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost
> >>>>>> all free
> >>>>>> >> as well.
> >>>>>> >>
> >>>>>> >> I am thinking that this is DB related, given that it's pausing
> when
> >>>>>> >> executing an update. Was looking at the update_state method in
> >>>>>> models.py,
> >>>>>> >> which logs right before the 15s pause.
> >>>>>> >>
> >>>>>> >> Cheers,
> >>>>>> >> Chris
> >>>>>> >>
> >>>>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <
> >>>>>> bdbruin@gmail.com>
> >>>>>> >> wrote:
> >>>>>> >>
> >>>>>> >>> Hi Chris,
> >>>>>> >>>
> >>>>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
> >>>>>> executor
> >>>>>> >>> is faster in checking the state than the TaskInstance is able to
> >>>>>> exit
> >>>>>> >>> itself. No, I don’t consider it normal, but it was sometimes
> >>>>>> reported (
> >>>>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though
> >>>>>> not really
> >>>>>> >>> replicable as of yet.
> >>>>>> >>>
> >>>>>> >>> Parsing seems exceptionally slow, it might be worth looking at
> it
> >>>>>> with a
> >>>>>> >>> debugger. Very faint guess might be that something with the
> >>>>>> multiprocessing
> >>>>>> >>> part could do something with memory that is costly, but then we
> >>>>>> need to
> >>>>>> >>> know more about what is running on the system. Never clue of
> >>>>>> system metrics
> >>>>>> >>> could be helpful here.
> >>>>>> >>>
> >>>>>> >>> Bolke
> >>>>>> >>>
> >>>>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <
> criccomini@apache.org
> >>>>>> >
> >>>>>> >>> wrote:
> >>>>>> >>>>
> >>>>>> >>>> Also, seeing this in EVERY task that runs:
> >>>>>> >>>>
> >>>>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of
> this
> >>>>>> >>>> instance has been externally set to queued. Taking the poison
> >>>>>> pill. So
> >>>>>> >>>> long.
> >>>>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited
> with
> >>>>>> return
> >>>>>> >>> code 0
> >>>>>> >>>>
> >>>>>> >>>>
> >>>>>> >>>> All successful tasks are showing this at the end of their logs.
> >>>>>> Is this
> >>>>>> >>>> normal?
> >>>>>> >>>>
> >>>>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
> >>>>>> >>> criccomini@apache.org>
> >>>>>> >>>> wrote:
> >>>>>> >>>>
> >>>>>> >>>>> Hey all,
> >>>>>> >>>>>
> >>>>>> >>>>> I've upgraded on production. Things seem to be working so far
> >>>>>> (only
> >>>>>> >>> been
> >>>>>> >>>>> an hour), but I am seeing this in the scheduler logs:
> >>>>>> >>>>>
> >>>>>> >>>>> File Path
> >>>>>> >>> PID
> >>>>>> >>>>> Runtime    Last Runtime    Last Run
> >>>>>> >>>>> ------------------------------------------------------------
> >>>>>> ------
> >>>>>> >>> -----
> >>>>>> >>>>> ---------  --------------  -------------------
> >>>>>> >>>>> ...
> >>>>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
> >>>>>> >>> 24793
> >>>>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
> >>>>>> >>>>> ...
> >>>>>> >>>>>
> >>>>>> >>>>> It seems to be taking more than 15 minutes to parse this DAG.
> >>>>>> Any idea
> >>>>>> >>>>> what's causing this? Scheduler config:
> >>>>>> >>>>>
> >>>>>> >>>>> [scheduler]
> >>>>>> >>>>> job_heartbeat_sec = 5
> >>>>>> >>>>> scheduler_heartbeat_sec = 5
> >>>>>> >>>>> max_threads = 2
> >>>>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
> >>>>>> >>>>>
> >>>>>> >>>>> The db.py file, itself, doesn't interact with any outside
> >>>>>> systems, so I
> >>>>>> >>>>> would have expected this to not take so long. It does,
> however,
> >>>>>> >>>>> programmatically generate many DAGs within the single .py
> file.
> >>>>>> >>>>>
> >>>>>> >>>>> A snippet of the scheduler log is here:
> >>>>>> >>>>>
> >>>>>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669
> >>>>>> e8ffd65
> >>>>>> >>>>>
> >>>>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea
> >>>>>> what's
> >>>>>> >>> going
> >>>>>> >>>>> on?
> >>>>>> >>>>>
> >>>>>> >>>>> Cheers,
> >>>>>> >>>>> Chris
> >>>>>> >>>>>
> >>>>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <
> >>>>>> bdbruin@gmail.com>
> >>>>>> >>> wrote:
> >>>>>> >>>>>
> >>>>>> >>>>>> I created:
> >>>>>> >>>>>>
> >>>>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
> >>>>>> checked, but
> >>>>>> >>>>>> not fixed
> >>>>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but
> >>>>>> don’t get
> >>>>>> >>> fixed
> >>>>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
> >>>>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong
> start_date /
> >>>>>> >>> interval
> >>>>>> >>>>>> was specified
> >>>>>> >>>>>>
> >>>>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out
> as
> >>>>>> a PR.
> >>>>>> >>>>>>
> >>>>>> >>>>>> Please note that I don't consider any of these blockers for a
> >>>>>> release
> >>>>>> >>> of
> >>>>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track
> for
> >>>>>> an RC
> >>>>>> >>> on Feb
> >>>>>> >>>>>> 2. However if people are using a restarting scheduler
> >>>>>> (run_duration
> >>>>>> >>> is set)
> >>>>>> >>>>>> and have a lot of running dag runs they won’t like
> >>>>>> AIRFLOW-791. So a
> >>>>>> >>>>>> workaround for this would be nice (we just updated dag_runs
> >>>>>> directly
> >>>>>> >>> in the
> >>>>>> >>>>>> database to say ‘finished’ before a certain date, but we are
> >>>>>> also not
> >>>>>> >>> using
> >>>>>> >>>>>> the run_duration).
> >>>>>> >>>>>>
> >>>>>> >>>>>> Bolke
> >>>>>> >>>>>>
> >>>>>> >>>>>>
> >>>>>> >>>>>>
> >>>>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bdbruin@gmail.com
> >
> >>>>>> wrote:
> >>>>>> >>>>>>>
> >>>>>> >>>>>>> Will do. And thanks.
> >>>>>> >>>>>>>
> >>>>>> >>>>>>> Adding another issue:
> >>>>>> >>>>>>>
> >>>>>> >>>>>>> * Some of our DAGs are not getting scheduled for some
> unknown
> >>>>>> reason.
> >>>>>> >>>>>>> Need to investigate why.
> >>>>>> >>>>>>>
> >>>>>> >>>>>>> Related but not root cause:
> >>>>>> >>>>>>> * Logging is so chatty that it gets really hard to find the
> >>>>>> real
> >>>>>> >>> issue
> >>>>>> >>>>>>>
> >>>>>> >>>>>>> Bolke.
> >>>>>> >>>>>>>
> >>>>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <
> >>>>>> dan.davydov@airbnb.com
> >>>>>> >>> .INVALID>
> >>>>>> >>>>>> wrote:
> >>>>>> >>>>>>>>
> >>>>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and
> >>>>>> hopefully some
> >>>>>> >>>>>> others
> >>>>>> >>>>>>>> are too. Do you mind creating jiras for these since you
> have
> >>>>>> the
> >>>>>> >>> full
> >>>>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it
> >>>>>> to
> >>>>>> >>> myself:
> >>>>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
> >>>>>> >>>>>>>>
> >>>>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
> >>>>>> bdbruin@gmail.com>
> >>>>>> >>>>>> wrote:
> >>>>>> >>>>>>>>
> >>>>>> >>>>>>>>> This is to report back on some of the (early) experiences
> >>>>>> we have
> >>>>>> >>> with
> >>>>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion
> for
> >>>>>> >>>>>> developers.
> >>>>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI
> would
> >>>>>> report
> >>>>>> >>> a
> >>>>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising
> >>>>>> (but not
> >>>>>> >>>>>>>>> reporting back errors)
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >>>>>> >>>>>>>>> We run in a secure environment which requires this
> variable
> >>>>>> to be
> >>>>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
> >>>>>> UPDATING.md)
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get
> >>>>>> fixed
> >>>>>> >>>>>>>>> Log gets flooded without a suggestion what to do
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
> >>>>>> seemed to
> >>>>>> >>>>>> have a
> >>>>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
> >>>>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for
> every
> >>>>>> log
> >>>>>> >>>>>> message
> >>>>>> >>>>>>>>> making it very slow
> >>>>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’
> >>>>>> states were
> >>>>>> >>>>>> not
> >>>>>> >>>>>>>>> being updated
> >>>>>> >>>>>>>>> - These dag_runs would never er be marked anything else
> than
> >>>>>> >>> running
> >>>>>> >>>>>> for
> >>>>>> >>>>>>>>> some reason
> >>>>>> >>>>>>>>> -> Applied work around to update all dag_run in sql
> before a
> >>>>>> >>> certain
> >>>>>> >>>>>> date
> >>>>>> >>>>>>>>> to -> finished
> >>>>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
> >>>>>> >>>>>> “finished/failed”
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>> 5. Our umask is set to 027
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>>>
> >>>>>> >>>>>>>
> >>>>>> >>>>>>
> >>>>>> >>>>>>
> >>>>>> >>>>>
> >>>>>> >>>
> >>>>>> >>>
> >>>>>> >>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
With this patch:

$ git diff
diff --git a/airflow/jobs.py b/airflow/jobs.py
index f1de333..9d08e75 100644
--- a/airflow/jobs.py
+++ b/airflow/jobs.py
@@ -544,6 +544,7 @@ class SchedulerJob(BaseJob):
             .query(
                 TI.task_id,
                 func.max(TI.execution_date).label('max_ti'))
+            .with_hint(TI, 'USE INDEX (PRIMARY)')
             .filter(TI.dag_id == dag.dag_id)
             .filter(TI.state == State.SUCCESS)
             .filter(TI.task_id.in_(dag.task_ids))

The db.py file parses in 90s. About 10x faster.

Th slow_query log is not showing any offending queries anymore. I will open
a JIRA for this. Note that with_hint stuff I put in is MySQL specific, I
think. Any one have any other ideas on how to affect the equivalent outcome?

On Mon, Jan 23, 2017 at 3:08 PM, Chris Riccomini <cr...@apache.org>
wrote:

> OK, it's using `state` instead of PRIMARY. Using PRIMARY with a hint,
> query takes .47s. Without hint, 10s. Going to try and patch.
>
> On Mon, Jan 23, 2017 at 2:57 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> This inner query takes 10s:
>>
>> SELECT task_instance.task_id AS task_id, max(task_instance.execution_date)
>> AS max_ti
>> FROM task_instance
>> WHERE task_instance.dag_id = 'dag1' AND task_instance.state = 'success'
>> AND task_instance.task_id IN ('t1', 't2') GROUP BY task_instance.task_id
>>
>> Explain seems OK:
>>
>> +----+-------------+---------------+------+-----------------
>> -----------------------------------+----------+---------+---
>> ----+-------+--------------------------+
>> | id | select_type | table         | type | possible_keys
>>                      | key      | key_len | ref   | rows  | Extra
>>          |
>> +----+-------------+---------------+------+-----------------
>> -----------------------------------+----------+---------+---
>> ----+-------+--------------------------+
>> |  1 | SIMPLE      | task_instance | ref  | PRIMARY,ti_dag_state,ti_pool,ti_state_lkp,ti_state
>> | ti_state | 63      | const | 81898 | Using where; Using index |
>> +----+-------------+---------------+------+-----------------
>> -----------------------------------+----------+---------+---
>> ----+-------+--------------------------+
>>
>> A look at the query indicates that it's using the state field as its
>> index lookup, which isn't good.
>>
>> On Mon, Jan 23, 2017 at 2:49 PM, Chris Riccomini <cr...@apache.org>
>> wrote:
>>
>>> It's this query:
>>>
>>> SELECT task_instance.task_id AS task_instance_task_id,
>>> task_instance.dag_id AS task_instance_dag_id, task_instance.execution_date
>>> AS task_instance_execution_date, task_instance.start_date AS
>>> task_instance_start_date, task_instance.end_date AS task_instance_end_date,
>>> task_instance.duration AS task_instance_duration, task_instance.state AS
>>> task_instance_state, task_instance.try_number AS task_instance_try_number,
>>> task_instance.hostname AS task_instance_hostname, task_instance.unixname AS
>>> task_instance_unixname, task_instance.job_id AS task_instance_job_id,
>>> task_instance.pool AS task_instance_pool, task_instance.queue AS
>>> task_instance_queue, task_instance.priority_weight AS
>>> task_instance_priority_weight, task_instance.operator AS
>>> task_instance_operator, task_instance.queued_dttm AS
>>> task_instance_queued_dttm, task_instance.pid AS task_instance_pid
>>> FROM task_instance, (SELECT task_instance.task_id AS task_id,
>>> max(task_instance.execution_date) AS max_ti
>>> FROM task_instance
>>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
>>> 'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5', 't6')
>>> GROUP BY task_instance.task_id) AS sq
>>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id =
>>> sq.task_id AND task_instance.execution_date = sq.max_ti
>>>
>>> Taking 10-20s
>>>
>>> On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <cr...@apache.org>
>>> wrote:
>>>
>>>> Can confirm it's a slow query on task_instance table. Still digging.
>>>> Unfortunately, the query is truncated in my UI right now:
>>>>
>>>> SELECT task_instance.task_id AS task_instance_...
>>>>
>>>> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <criccomini@apache.org
>>>> > wrote:
>>>>
>>>>> Digging. Might be a bit.
>>>>>
>>>>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Slow query log? Db load?
>>>>>>
>>>>>> B.
>>>>>>
>>>>>> Verstuurd vanaf mijn iPad
>>>>>>
>>>>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <
>>>>>> criccomini@apache.org> het volgende geschreven:
>>>>>> >
>>>>>> > Note: 6.5 million TIs in the task_instance table.
>>>>>> >
>>>>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
>>>>>> criccomini@apache.org>
>>>>>> > wrote:
>>>>>> >
>>>>>> >> Hey Bolke,
>>>>>> >>
>>>>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost
>>>>>> all free
>>>>>> >> as well.
>>>>>> >>
>>>>>> >> I am thinking that this is DB related, given that it's pausing when
>>>>>> >> executing an update. Was looking at the update_state method in
>>>>>> models.py,
>>>>>> >> which logs right before the 15s pause.
>>>>>> >>
>>>>>> >> Cheers,
>>>>>> >> Chris
>>>>>> >>
>>>>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <
>>>>>> bdbruin@gmail.com>
>>>>>> >> wrote:
>>>>>> >>
>>>>>> >>> Hi Chris,
>>>>>> >>>
>>>>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
>>>>>> executor
>>>>>> >>> is faster in checking the state than the TaskInstance is able to
>>>>>> exit
>>>>>> >>> itself. No, I don’t consider it normal, but it was sometimes
>>>>>> reported (
>>>>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though
>>>>>> not really
>>>>>> >>> replicable as of yet.
>>>>>> >>>
>>>>>> >>> Parsing seems exceptionally slow, it might be worth looking at it
>>>>>> with a
>>>>>> >>> debugger. Very faint guess might be that something with the
>>>>>> multiprocessing
>>>>>> >>> part could do something with memory that is costly, but then we
>>>>>> need to
>>>>>> >>> know more about what is running on the system. Never clue of
>>>>>> system metrics
>>>>>> >>> could be helpful here.
>>>>>> >>>
>>>>>> >>> Bolke
>>>>>> >>>
>>>>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <criccomini@apache.org
>>>>>> >
>>>>>> >>> wrote:
>>>>>> >>>>
>>>>>> >>>> Also, seeing this in EVERY task that runs:
>>>>>> >>>>
>>>>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>>>>>> >>>> instance has been externally set to queued. Taking the poison
>>>>>> pill. So
>>>>>> >>>> long.
>>>>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>>>>>> return
>>>>>> >>> code 0
>>>>>> >>>>
>>>>>> >>>>
>>>>>> >>>> All successful tasks are showing this at the end of their logs.
>>>>>> Is this
>>>>>> >>>> normal?
>>>>>> >>>>
>>>>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>>>>>> >>> criccomini@apache.org>
>>>>>> >>>> wrote:
>>>>>> >>>>
>>>>>> >>>>> Hey all,
>>>>>> >>>>>
>>>>>> >>>>> I've upgraded on production. Things seem to be working so far
>>>>>> (only
>>>>>> >>> been
>>>>>> >>>>> an hour), but I am seeing this in the scheduler logs:
>>>>>> >>>>>
>>>>>> >>>>> File Path
>>>>>> >>> PID
>>>>>> >>>>> Runtime    Last Runtime    Last Run
>>>>>> >>>>> ------------------------------------------------------------
>>>>>> ------
>>>>>> >>> -----
>>>>>> >>>>> ---------  --------------  -------------------
>>>>>> >>>>> ...
>>>>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
>>>>>> >>> 24793
>>>>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
>>>>>> >>>>> ...
>>>>>> >>>>>
>>>>>> >>>>> It seems to be taking more than 15 minutes to parse this DAG.
>>>>>> Any idea
>>>>>> >>>>> what's causing this? Scheduler config:
>>>>>> >>>>>
>>>>>> >>>>> [scheduler]
>>>>>> >>>>> job_heartbeat_sec = 5
>>>>>> >>>>> scheduler_heartbeat_sec = 5
>>>>>> >>>>> max_threads = 2
>>>>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
>>>>>> >>>>>
>>>>>> >>>>> The db.py file, itself, doesn't interact with any outside
>>>>>> systems, so I
>>>>>> >>>>> would have expected this to not take so long. It does, however,
>>>>>> >>>>> programmatically generate many DAGs within the single .py file.
>>>>>> >>>>>
>>>>>> >>>>> A snippet of the scheduler log is here:
>>>>>> >>>>>
>>>>>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669
>>>>>> e8ffd65
>>>>>> >>>>>
>>>>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea
>>>>>> what's
>>>>>> >>> going
>>>>>> >>>>> on?
>>>>>> >>>>>
>>>>>> >>>>> Cheers,
>>>>>> >>>>> Chris
>>>>>> >>>>>
>>>>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <
>>>>>> bdbruin@gmail.com>
>>>>>> >>> wrote:
>>>>>> >>>>>
>>>>>> >>>>>> I created:
>>>>>> >>>>>>
>>>>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
>>>>>> checked, but
>>>>>> >>>>>> not fixed
>>>>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but
>>>>>> don’t get
>>>>>> >>> fixed
>>>>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>>>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>>>>>> >>> interval
>>>>>> >>>>>> was specified
>>>>>> >>>>>>
>>>>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as
>>>>>> a PR.
>>>>>> >>>>>>
>>>>>> >>>>>> Please note that I don't consider any of these blockers for a
>>>>>> release
>>>>>> >>> of
>>>>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for
>>>>>> an RC
>>>>>> >>> on Feb
>>>>>> >>>>>> 2. However if people are using a restarting scheduler
>>>>>> (run_duration
>>>>>> >>> is set)
>>>>>> >>>>>> and have a lot of running dag runs they won’t like
>>>>>> AIRFLOW-791. So a
>>>>>> >>>>>> workaround for this would be nice (we just updated dag_runs
>>>>>> directly
>>>>>> >>> in the
>>>>>> >>>>>> database to say ‘finished’ before a certain date, but we are
>>>>>> also not
>>>>>> >>> using
>>>>>> >>>>>> the run_duration).
>>>>>> >>>>>>
>>>>>> >>>>>> Bolke
>>>>>> >>>>>>
>>>>>> >>>>>>
>>>>>> >>>>>>
>>>>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
>>>>>> wrote:
>>>>>> >>>>>>>
>>>>>> >>>>>>> Will do. And thanks.
>>>>>> >>>>>>>
>>>>>> >>>>>>> Adding another issue:
>>>>>> >>>>>>>
>>>>>> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
>>>>>> reason.
>>>>>> >>>>>>> Need to investigate why.
>>>>>> >>>>>>>
>>>>>> >>>>>>> Related but not root cause:
>>>>>> >>>>>>> * Logging is so chatty that it gets really hard to find the
>>>>>> real
>>>>>> >>> issue
>>>>>> >>>>>>>
>>>>>> >>>>>>> Bolke.
>>>>>> >>>>>>>
>>>>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <
>>>>>> dan.davydov@airbnb.com
>>>>>> >>> .INVALID>
>>>>>> >>>>>> wrote:
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and
>>>>>> hopefully some
>>>>>> >>>>>> others
>>>>>> >>>>>>>> are too. Do you mind creating jiras for these since you have
>>>>>> the
>>>>>> >>> full
>>>>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it
>>>>>> to
>>>>>> >>> myself:
>>>>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
>>>>>> bdbruin@gmail.com>
>>>>>> >>>>>> wrote:
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>> This is to report back on some of the (early) experiences
>>>>>> we have
>>>>>> >>> with
>>>>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>>>>> >>>>>> developers.
>>>>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
>>>>>> report
>>>>>> >>> a
>>>>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising
>>>>>> (but not
>>>>>> >>>>>>>>> reporting back errors)
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>>> >>>>>>>>> We run in a secure environment which requires this variable
>>>>>> to be
>>>>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
>>>>>> UPDATING.md)
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get
>>>>>> fixed
>>>>>> >>>>>>>>> Log gets flooded without a suggestion what to do
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
>>>>>> seemed to
>>>>>> >>>>>> have a
>>>>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every
>>>>>> log
>>>>>> >>>>>> message
>>>>>> >>>>>>>>> making it very slow
>>>>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’
>>>>>> states were
>>>>>> >>>>>> not
>>>>>> >>>>>>>>> being updated
>>>>>> >>>>>>>>> - These dag_runs would never er be marked anything else than
>>>>>> >>> running
>>>>>> >>>>>> for
>>>>>> >>>>>>>>> some reason
>>>>>> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
>>>>>> >>> certain
>>>>>> >>>>>> date
>>>>>> >>>>>>>>> to -> finished
>>>>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
>>>>>> >>>>>> “finished/failed”
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>> 5. Our umask is set to 027
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>>>
>>>>>> >>>>>>>
>>>>>> >>>>>>
>>>>>> >>>>>>
>>>>>> >>>>>
>>>>>> >>>
>>>>>> >>>
>>>>>> >>
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
OK, it's using `state` instead of PRIMARY. Using PRIMARY with a hint, query
takes .47s. Without hint, 10s. Going to try and patch.

On Mon, Jan 23, 2017 at 2:57 PM, Chris Riccomini <cr...@apache.org>
wrote:

> This inner query takes 10s:
>
> SELECT task_instance.task_id AS task_id, max(task_instance.execution_date)
> AS max_ti
> FROM task_instance
> WHERE task_instance.dag_id = 'dag1' AND task_instance.state = 'success'
> AND task_instance.task_id IN ('t1', 't2') GROUP BY task_instance.task_id
>
> Explain seems OK:
>
> +----+-------------+---------------+------+-----------------
> -----------------------------------+----------+---------+---
> ----+-------+--------------------------+
> | id | select_type | table         | type | possible_keys
>                      | key      | key_len | ref   | rows  | Extra
>          |
> +----+-------------+---------------+------+-----------------
> -----------------------------------+----------+---------+---
> ----+-------+--------------------------+
> |  1 | SIMPLE      | task_instance | ref  | PRIMARY,ti_dag_state,ti_pool,ti_state_lkp,ti_state
> | ti_state | 63      | const | 81898 | Using where; Using index |
> +----+-------------+---------------+------+-----------------
> -----------------------------------+----------+---------+---
> ----+-------+--------------------------+
>
> A look at the query indicates that it's using the state field as its index
> lookup, which isn't good.
>
> On Mon, Jan 23, 2017 at 2:49 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> It's this query:
>>
>> SELECT task_instance.task_id AS task_instance_task_id,
>> task_instance.dag_id AS task_instance_dag_id, task_instance.execution_date
>> AS task_instance_execution_date, task_instance.start_date AS
>> task_instance_start_date, task_instance.end_date AS task_instance_end_date,
>> task_instance.duration AS task_instance_duration, task_instance.state AS
>> task_instance_state, task_instance.try_number AS task_instance_try_number,
>> task_instance.hostname AS task_instance_hostname, task_instance.unixname AS
>> task_instance_unixname, task_instance.job_id AS task_instance_job_id,
>> task_instance.pool AS task_instance_pool, task_instance.queue AS
>> task_instance_queue, task_instance.priority_weight AS
>> task_instance_priority_weight, task_instance.operator AS
>> task_instance_operator, task_instance.queued_dttm AS
>> task_instance_queued_dttm, task_instance.pid AS task_instance_pid
>> FROM task_instance, (SELECT task_instance.task_id AS task_id,
>> max(task_instance.execution_date) AS max_ti
>> FROM task_instance
>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
>> 'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5', 't6')
>> GROUP BY task_instance.task_id) AS sq
>> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id =
>> sq.task_id AND task_instance.execution_date = sq.max_ti
>>
>> Taking 10-20s
>>
>> On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <cr...@apache.org>
>> wrote:
>>
>>> Can confirm it's a slow query on task_instance table. Still digging.
>>> Unfortunately, the query is truncated in my UI right now:
>>>
>>> SELECT task_instance.task_id AS task_instance_...
>>>
>>> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <cr...@apache.org>
>>> wrote:
>>>
>>>> Digging. Might be a bit.
>>>>
>>>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com>
>>>> wrote:
>>>>
>>>>> Slow query log? Db load?
>>>>>
>>>>> B.
>>>>>
>>>>> Verstuurd vanaf mijn iPad
>>>>>
>>>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <
>>>>> criccomini@apache.org> het volgende geschreven:
>>>>> >
>>>>> > Note: 6.5 million TIs in the task_instance table.
>>>>> >
>>>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
>>>>> criccomini@apache.org>
>>>>> > wrote:
>>>>> >
>>>>> >> Hey Bolke,
>>>>> >>
>>>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost
>>>>> all free
>>>>> >> as well.
>>>>> >>
>>>>> >> I am thinking that this is DB related, given that it's pausing when
>>>>> >> executing an update. Was looking at the update_state method in
>>>>> models.py,
>>>>> >> which logs right before the 15s pause.
>>>>> >>
>>>>> >> Cheers,
>>>>> >> Chris
>>>>> >>
>>>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bdbruin@gmail.com
>>>>> >
>>>>> >> wrote:
>>>>> >>
>>>>> >>> Hi Chris,
>>>>> >>>
>>>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
>>>>> executor
>>>>> >>> is faster in checking the state than the TaskInstance is able to
>>>>> exit
>>>>> >>> itself. No, I don’t consider it normal, but it was sometimes
>>>>> reported (
>>>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though not
>>>>> really
>>>>> >>> replicable as of yet.
>>>>> >>>
>>>>> >>> Parsing seems exceptionally slow, it might be worth looking at it
>>>>> with a
>>>>> >>> debugger. Very faint guess might be that something with the
>>>>> multiprocessing
>>>>> >>> part could do something with memory that is costly, but then we
>>>>> need to
>>>>> >>> know more about what is running on the system. Never clue of
>>>>> system metrics
>>>>> >>> could be helpful here.
>>>>> >>>
>>>>> >>> Bolke
>>>>> >>>
>>>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>>>>> >>> wrote:
>>>>> >>>>
>>>>> >>>> Also, seeing this in EVERY task that runs:
>>>>> >>>>
>>>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>>>>> >>>> instance has been externally set to queued. Taking the poison
>>>>> pill. So
>>>>> >>>> long.
>>>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>>>>> return
>>>>> >>> code 0
>>>>> >>>>
>>>>> >>>>
>>>>> >>>> All successful tasks are showing this at the end of their logs.
>>>>> Is this
>>>>> >>>> normal?
>>>>> >>>>
>>>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>>>>> >>> criccomini@apache.org>
>>>>> >>>> wrote:
>>>>> >>>>
>>>>> >>>>> Hey all,
>>>>> >>>>>
>>>>> >>>>> I've upgraded on production. Things seem to be working so far
>>>>> (only
>>>>> >>> been
>>>>> >>>>> an hour), but I am seeing this in the scheduler logs:
>>>>> >>>>>
>>>>> >>>>> File Path
>>>>> >>> PID
>>>>> >>>>> Runtime    Last Runtime    Last Run
>>>>> >>>>> ------------------------------------------------------------
>>>>> ------
>>>>> >>> -----
>>>>> >>>>> ---------  --------------  -------------------
>>>>> >>>>> ...
>>>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
>>>>> >>> 24793
>>>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
>>>>> >>>>> ...
>>>>> >>>>>
>>>>> >>>>> It seems to be taking more than 15 minutes to parse this DAG.
>>>>> Any idea
>>>>> >>>>> what's causing this? Scheduler config:
>>>>> >>>>>
>>>>> >>>>> [scheduler]
>>>>> >>>>> job_heartbeat_sec = 5
>>>>> >>>>> scheduler_heartbeat_sec = 5
>>>>> >>>>> max_threads = 2
>>>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
>>>>> >>>>>
>>>>> >>>>> The db.py file, itself, doesn't interact with any outside
>>>>> systems, so I
>>>>> >>>>> would have expected this to not take so long. It does, however,
>>>>> >>>>> programmatically generate many DAGs within the single .py file.
>>>>> >>>>>
>>>>> >>>>> A snippet of the scheduler log is here:
>>>>> >>>>>
>>>>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669
>>>>> e8ffd65
>>>>> >>>>>
>>>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea
>>>>> what's
>>>>> >>> going
>>>>> >>>>> on?
>>>>> >>>>>
>>>>> >>>>> Cheers,
>>>>> >>>>> Chris
>>>>> >>>>>
>>>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <
>>>>> bdbruin@gmail.com>
>>>>> >>> wrote:
>>>>> >>>>>
>>>>> >>>>>> I created:
>>>>> >>>>>>
>>>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
>>>>> checked, but
>>>>> >>>>>> not fixed
>>>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but
>>>>> don’t get
>>>>> >>> fixed
>>>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>>>>> >>> interval
>>>>> >>>>>> was specified
>>>>> >>>>>>
>>>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as
>>>>> a PR.
>>>>> >>>>>>
>>>>> >>>>>> Please note that I don't consider any of these blockers for a
>>>>> release
>>>>> >>> of
>>>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for
>>>>> an RC
>>>>> >>> on Feb
>>>>> >>>>>> 2. However if people are using a restarting scheduler
>>>>> (run_duration
>>>>> >>> is set)
>>>>> >>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791.
>>>>> So a
>>>>> >>>>>> workaround for this would be nice (we just updated dag_runs
>>>>> directly
>>>>> >>> in the
>>>>> >>>>>> database to say ‘finished’ before a certain date, but we are
>>>>> also not
>>>>> >>> using
>>>>> >>>>>> the run_duration).
>>>>> >>>>>>
>>>>> >>>>>> Bolke
>>>>> >>>>>>
>>>>> >>>>>>
>>>>> >>>>>>
>>>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
>>>>> wrote:
>>>>> >>>>>>>
>>>>> >>>>>>> Will do. And thanks.
>>>>> >>>>>>>
>>>>> >>>>>>> Adding another issue:
>>>>> >>>>>>>
>>>>> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
>>>>> reason.
>>>>> >>>>>>> Need to investigate why.
>>>>> >>>>>>>
>>>>> >>>>>>> Related but not root cause:
>>>>> >>>>>>> * Logging is so chatty that it gets really hard to find the
>>>>> real
>>>>> >>> issue
>>>>> >>>>>>>
>>>>> >>>>>>> Bolke.
>>>>> >>>>>>>
>>>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>>>>> >>> .INVALID>
>>>>> >>>>>> wrote:
>>>>> >>>>>>>>
>>>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully
>>>>> some
>>>>> >>>>>> others
>>>>> >>>>>>>> are too. Do you mind creating jiras for these since you have
>>>>> the
>>>>> >>> full
>>>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it to
>>>>> >>> myself:
>>>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>> >>>>>>>>
>>>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
>>>>> bdbruin@gmail.com>
>>>>> >>>>>> wrote:
>>>>> >>>>>>>>
>>>>> >>>>>>>>> This is to report back on some of the (early) experiences we
>>>>> have
>>>>> >>> with
>>>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>> >>>>>>>>>
>>>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>>>> >>>>>> developers.
>>>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
>>>>> report
>>>>> >>> a
>>>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising
>>>>> (but not
>>>>> >>>>>>>>> reporting back errors)
>>>>> >>>>>>>>>
>>>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>> >>>>>>>>> We run in a secure environment which requires this variable
>>>>> to be
>>>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
>>>>> UPDATING.md)
>>>>> >>>>>>>>>
>>>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get
>>>>> fixed
>>>>> >>>>>>>>> Log gets flooded without a suggestion what to do
>>>>> >>>>>>>>>
>>>>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
>>>>> seemed to
>>>>> >>>>>> have a
>>>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every
>>>>> log
>>>>> >>>>>> message
>>>>> >>>>>>>>> making it very slow
>>>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’
>>>>> states were
>>>>> >>>>>> not
>>>>> >>>>>>>>> being updated
>>>>> >>>>>>>>> - These dag_runs would never er be marked anything else than
>>>>> >>> running
>>>>> >>>>>> for
>>>>> >>>>>>>>> some reason
>>>>> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
>>>>> >>> certain
>>>>> >>>>>> date
>>>>> >>>>>>>>> to -> finished
>>>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
>>>>> >>>>>> “finished/failed”
>>>>> >>>>>>>>>
>>>>> >>>>>>>>> 5. Our umask is set to 027
>>>>> >>>>>>>>>
>>>>> >>>>>>>>>
>>>>> >>>>>>>
>>>>> >>>>>>
>>>>> >>>>>>
>>>>> >>>>>
>>>>> >>>
>>>>> >>>
>>>>> >>
>>>>>
>>>>
>>>>
>>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
This inner query takes 10s:

SELECT task_instance.task_id AS task_id, max(task_instance.execution_date)
AS max_ti
FROM task_instance
WHERE task_instance.dag_id = 'dag1' AND task_instance.state = 'success' AND
task_instance.task_id IN ('t1', 't2') GROUP BY task_instance.task_id

Explain seems OK:

+----+-------------+---------------+------+----------------------------------------------------+----------+---------+-------+-------+--------------------------+
| id | select_type | table         | type | possible_keys
                   | key      | key_len | ref   | rows  | Extra
       |
+----+-------------+---------------+------+----------------------------------------------------+----------+---------+-------+-------+--------------------------+
|  1 | SIMPLE      | task_instance | ref  |
PRIMARY,ti_dag_state,ti_pool,ti_state_lkp,ti_state | ti_state | 63      |
const | 81898 | Using where; Using index |
+----+-------------+---------------+------+----------------------------------------------------+----------+---------+-------+-------+--------------------------+

A look at the query indicates that it's using the state field as its index
lookup, which isn't good.

On Mon, Jan 23, 2017 at 2:49 PM, Chris Riccomini <cr...@apache.org>
wrote:

> It's this query:
>
> SELECT task_instance.task_id AS task_instance_task_id,
> task_instance.dag_id AS task_instance_dag_id, task_instance.execution_date
> AS task_instance_execution_date, task_instance.start_date AS
> task_instance_start_date, task_instance.end_date AS task_instance_end_date,
> task_instance.duration AS task_instance_duration, task_instance.state AS
> task_instance_state, task_instance.try_number AS task_instance_try_number,
> task_instance.hostname AS task_instance_hostname, task_instance.unixname AS
> task_instance_unixname, task_instance.job_id AS task_instance_job_id,
> task_instance.pool AS task_instance_pool, task_instance.queue AS
> task_instance_queue, task_instance.priority_weight AS
> task_instance_priority_weight, task_instance.operator AS
> task_instance_operator, task_instance.queued_dttm AS
> task_instance_queued_dttm, task_instance.pid AS task_instance_pid
> FROM task_instance, (SELECT task_instance.task_id AS task_id,
> max(task_instance.execution_date) AS max_ti
> FROM task_instance
> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
> 'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5', 't6')
> GROUP BY task_instance.task_id) AS sq
> WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id =
> sq.task_id AND task_instance.execution_date = sq.max_ti
>
> Taking 10-20s
>
> On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Can confirm it's a slow query on task_instance table. Still digging.
>> Unfortunately, the query is truncated in my UI right now:
>>
>> SELECT task_instance.task_id AS task_instance_...
>>
>> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <cr...@apache.org>
>> wrote:
>>
>>> Digging. Might be a bit.
>>>
>>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com>
>>> wrote:
>>>
>>>> Slow query log? Db load?
>>>>
>>>> B.
>>>>
>>>> Verstuurd vanaf mijn iPad
>>>>
>>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <cr...@apache.org>
>>>> het volgende geschreven:
>>>> >
>>>> > Note: 6.5 million TIs in the task_instance table.
>>>> >
>>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
>>>> criccomini@apache.org>
>>>> > wrote:
>>>> >
>>>> >> Hey Bolke,
>>>> >>
>>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all
>>>> free
>>>> >> as well.
>>>> >>
>>>> >> I am thinking that this is DB related, given that it's pausing when
>>>> >> executing an update. Was looking at the update_state method in
>>>> models.py,
>>>> >> which logs right before the 15s pause.
>>>> >>
>>>> >> Cheers,
>>>> >> Chris
>>>> >>
>>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
>>>> >> wrote:
>>>> >>
>>>> >>> Hi Chris,
>>>> >>>
>>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
>>>> executor
>>>> >>> is faster in checking the state than the TaskInstance is able to
>>>> exit
>>>> >>> itself. No, I don’t consider it normal, but it was sometimes
>>>> reported (
>>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though not
>>>> really
>>>> >>> replicable as of yet.
>>>> >>>
>>>> >>> Parsing seems exceptionally slow, it might be worth looking at it
>>>> with a
>>>> >>> debugger. Very faint guess might be that something with the
>>>> multiprocessing
>>>> >>> part could do something with memory that is costly, but then we
>>>> need to
>>>> >>> know more about what is running on the system. Never clue of system
>>>> metrics
>>>> >>> could be helpful here.
>>>> >>>
>>>> >>> Bolke
>>>> >>>
>>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>>>> >>> wrote:
>>>> >>>>
>>>> >>>> Also, seeing this in EVERY task that runs:
>>>> >>>>
>>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>>>> >>>> instance has been externally set to queued. Taking the poison
>>>> pill. So
>>>> >>>> long.
>>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>>>> return
>>>> >>> code 0
>>>> >>>>
>>>> >>>>
>>>> >>>> All successful tasks are showing this at the end of their logs. Is
>>>> this
>>>> >>>> normal?
>>>> >>>>
>>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>>>> >>> criccomini@apache.org>
>>>> >>>> wrote:
>>>> >>>>
>>>> >>>>> Hey all,
>>>> >>>>>
>>>> >>>>> I've upgraded on production. Things seem to be working so far
>>>> (only
>>>> >>> been
>>>> >>>>> an hour), but I am seeing this in the scheduler logs:
>>>> >>>>>
>>>> >>>>> File Path
>>>> >>> PID
>>>> >>>>> Runtime    Last Runtime    Last Run
>>>> >>>>> ------------------------------------------------------------
>>>> ------
>>>> >>> -----
>>>> >>>>> ---------  --------------  -------------------
>>>> >>>>> ...
>>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
>>>> >>> 24793
>>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
>>>> >>>>> ...
>>>> >>>>>
>>>> >>>>> It seems to be taking more than 15 minutes to parse this DAG. Any
>>>> idea
>>>> >>>>> what's causing this? Scheduler config:
>>>> >>>>>
>>>> >>>>> [scheduler]
>>>> >>>>> job_heartbeat_sec = 5
>>>> >>>>> scheduler_heartbeat_sec = 5
>>>> >>>>> max_threads = 2
>>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
>>>> >>>>>
>>>> >>>>> The db.py file, itself, doesn't interact with any outside
>>>> systems, so I
>>>> >>>>> would have expected this to not take so long. It does, however,
>>>> >>>>> programmatically generate many DAGs within the single .py file.
>>>> >>>>>
>>>> >>>>> A snippet of the scheduler log is here:
>>>> >>>>>
>>>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669
>>>> e8ffd65
>>>> >>>>>
>>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea what's
>>>> >>> going
>>>> >>>>> on?
>>>> >>>>>
>>>> >>>>> Cheers,
>>>> >>>>> Chris
>>>> >>>>>
>>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <
>>>> bdbruin@gmail.com>
>>>> >>> wrote:
>>>> >>>>>
>>>> >>>>>> I created:
>>>> >>>>>>
>>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
>>>> checked, but
>>>> >>>>>> not fixed
>>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t
>>>> get
>>>> >>> fixed
>>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>>>> >>> interval
>>>> >>>>>> was specified
>>>> >>>>>>
>>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a
>>>> PR.
>>>> >>>>>>
>>>> >>>>>> Please note that I don't consider any of these blockers for a
>>>> release
>>>> >>> of
>>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for
>>>> an RC
>>>> >>> on Feb
>>>> >>>>>> 2. However if people are using a restarting scheduler
>>>> (run_duration
>>>> >>> is set)
>>>> >>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791.
>>>> So a
>>>> >>>>>> workaround for this would be nice (we just updated dag_runs
>>>> directly
>>>> >>> in the
>>>> >>>>>> database to say ‘finished’ before a certain date, but we are
>>>> also not
>>>> >>> using
>>>> >>>>>> the run_duration).
>>>> >>>>>>
>>>> >>>>>> Bolke
>>>> >>>>>>
>>>> >>>>>>
>>>> >>>>>>
>>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
>>>> wrote:
>>>> >>>>>>>
>>>> >>>>>>> Will do. And thanks.
>>>> >>>>>>>
>>>> >>>>>>> Adding another issue:
>>>> >>>>>>>
>>>> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
>>>> reason.
>>>> >>>>>>> Need to investigate why.
>>>> >>>>>>>
>>>> >>>>>>> Related but not root cause:
>>>> >>>>>>> * Logging is so chatty that it gets really hard to find the real
>>>> >>> issue
>>>> >>>>>>>
>>>> >>>>>>> Bolke.
>>>> >>>>>>>
>>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>>>> >>> .INVALID>
>>>> >>>>>> wrote:
>>>> >>>>>>>>
>>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully
>>>> some
>>>> >>>>>> others
>>>> >>>>>>>> are too. Do you mind creating jiras for these since you have
>>>> the
>>>> >>> full
>>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it to
>>>> >>> myself:
>>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>> >>>>>>>>
>>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
>>>> bdbruin@gmail.com>
>>>> >>>>>> wrote:
>>>> >>>>>>>>
>>>> >>>>>>>>> This is to report back on some of the (early) experiences we
>>>> have
>>>> >>> with
>>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>> >>>>>>>>>
>>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>>> >>>>>> developers.
>>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
>>>> report
>>>> >>> a
>>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising
>>>> (but not
>>>> >>>>>>>>> reporting back errors)
>>>> >>>>>>>>>
>>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>> >>>>>>>>> We run in a secure environment which requires this variable
>>>> to be
>>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
>>>> UPDATING.md)
>>>> >>>>>>>>>
>>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>>> >>>>>>>>> Log gets flooded without a suggestion what to do
>>>> >>>>>>>>>
>>>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
>>>> seemed to
>>>> >>>>>> have a
>>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every
>>>> log
>>>> >>>>>> message
>>>> >>>>>>>>> making it very slow
>>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’
>>>> states were
>>>> >>>>>> not
>>>> >>>>>>>>> being updated
>>>> >>>>>>>>> - These dag_runs would never er be marked anything else than
>>>> >>> running
>>>> >>>>>> for
>>>> >>>>>>>>> some reason
>>>> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
>>>> >>> certain
>>>> >>>>>> date
>>>> >>>>>>>>> to -> finished
>>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
>>>> >>>>>> “finished/failed”
>>>> >>>>>>>>>
>>>> >>>>>>>>> 5. Our umask is set to 027
>>>> >>>>>>>>>
>>>> >>>>>>>>>
>>>> >>>>>>>
>>>> >>>>>>
>>>> >>>>>>
>>>> >>>>>
>>>> >>>
>>>> >>>
>>>> >>
>>>>
>>>
>>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
It's this query:

SELECT task_instance.task_id AS task_instance_task_id, task_instance.dag_id
AS task_instance_dag_id, task_instance.execution_date AS
task_instance_execution_date, task_instance.start_date AS
task_instance_start_date, task_instance.end_date AS task_instance_end_date,
task_instance.duration AS task_instance_duration, task_instance.state AS
task_instance_state, task_instance.try_number AS task_instance_try_number,
task_instance.hostname AS task_instance_hostname, task_instance.unixname AS
task_instance_unixname, task_instance.job_id AS task_instance_job_id,
task_instance.pool AS task_instance_pool, task_instance.queue AS
task_instance_queue, task_instance.priority_weight AS
task_instance_priority_weight, task_instance.operator AS
task_instance_operator, task_instance.queued_dttm AS
task_instance_queued_dttm, task_instance.pid AS task_instance_pid
FROM task_instance, (SELECT task_instance.task_id AS task_id,
max(task_instance.execution_date) AS max_ti
FROM task_instance
WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.state =
'success' AND task_instance.task_id IN ('t1', 't2', 't3', 't4', 't5', 't6')
GROUP BY task_instance.task_id) AS sq
WHERE task_instance.dag_id = 'some_dag_id' AND task_instance.task_id =
sq.task_id AND task_instance.execution_date = sq.max_ti

Taking 10-20s

On Mon, Jan 23, 2017 at 2:05 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Can confirm it's a slow query on task_instance table. Still digging.
> Unfortunately, the query is truncated in my UI right now:
>
> SELECT task_instance.task_id AS task_instance_...
>
> On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Digging. Might be a bit.
>>
>> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>>
>>> Slow query log? Db load?
>>>
>>> B.
>>>
>>> Verstuurd vanaf mijn iPad
>>>
>>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <cr...@apache.org>
>>> het volgende geschreven:
>>> >
>>> > Note: 6.5 million TIs in the task_instance table.
>>> >
>>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
>>> criccomini@apache.org>
>>> > wrote:
>>> >
>>> >> Hey Bolke,
>>> >>
>>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all
>>> free
>>> >> as well.
>>> >>
>>> >> I am thinking that this is DB related, given that it's pausing when
>>> >> executing an update. Was looking at the update_state method in
>>> models.py,
>>> >> which logs right before the 15s pause.
>>> >>
>>> >> Cheers,
>>> >> Chris
>>> >>
>>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
>>> >> wrote:
>>> >>
>>> >>> Hi Chris,
>>> >>>
>>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
>>> executor
>>> >>> is faster in checking the state than the TaskInstance is able to exit
>>> >>> itself. No, I don’t consider it normal, but it was sometimes
>>> reported (
>>> >>> https://github.com/apache/incubator-airflow/pull/1821) though not
>>> really
>>> >>> replicable as of yet.
>>> >>>
>>> >>> Parsing seems exceptionally slow, it might be worth looking at it
>>> with a
>>> >>> debugger. Very faint guess might be that something with the
>>> multiprocessing
>>> >>> part could do something with memory that is costly, but then we need
>>> to
>>> >>> know more about what is running on the system. Never clue of system
>>> metrics
>>> >>> could be helpful here.
>>> >>>
>>> >>> Bolke
>>> >>>
>>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>>> >>> wrote:
>>> >>>>
>>> >>>> Also, seeing this in EVERY task that runs:
>>> >>>>
>>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>>> >>>> instance has been externally set to queued. Taking the poison pill.
>>> So
>>> >>>> long.
>>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>>> return
>>> >>> code 0
>>> >>>>
>>> >>>>
>>> >>>> All successful tasks are showing this at the end of their logs. Is
>>> this
>>> >>>> normal?
>>> >>>>
>>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>>> >>> criccomini@apache.org>
>>> >>>> wrote:
>>> >>>>
>>> >>>>> Hey all,
>>> >>>>>
>>> >>>>> I've upgraded on production. Things seem to be working so far (only
>>> >>> been
>>> >>>>> an hour), but I am seeing this in the scheduler logs:
>>> >>>>>
>>> >>>>> File Path
>>> >>> PID
>>> >>>>> Runtime    Last Runtime    Last Run
>>> >>>>> ------------------------------------------------------------------
>>> >>> -----
>>> >>>>> ---------  --------------  -------------------
>>> >>>>> ...
>>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
>>> >>> 24793
>>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
>>> >>>>> ...
>>> >>>>>
>>> >>>>> It seems to be taking more than 15 minutes to parse this DAG. Any
>>> idea
>>> >>>>> what's causing this? Scheduler config:
>>> >>>>>
>>> >>>>> [scheduler]
>>> >>>>> job_heartbeat_sec = 5
>>> >>>>> scheduler_heartbeat_sec = 5
>>> >>>>> max_threads = 2
>>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
>>> >>>>>
>>> >>>>> The db.py file, itself, doesn't interact with any outside systems,
>>> so I
>>> >>>>> would have expected this to not take so long. It does, however,
>>> >>>>> programmatically generate many DAGs within the single .py file.
>>> >>>>>
>>> >>>>> A snippet of the scheduler log is here:
>>> >>>>>
>>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669
>>> e8ffd65
>>> >>>>>
>>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea what's
>>> >>> going
>>> >>>>> on?
>>> >>>>>
>>> >>>>> Cheers,
>>> >>>>> Chris
>>> >>>>>
>>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bdbruin@gmail.com
>>> >
>>> >>> wrote:
>>> >>>>>
>>> >>>>>> I created:
>>> >>>>>>
>>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being
>>> checked, but
>>> >>>>>> not fixed
>>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t
>>> get
>>> >>> fixed
>>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>>> >>> interval
>>> >>>>>> was specified
>>> >>>>>>
>>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a
>>> PR.
>>> >>>>>>
>>> >>>>>> Please note that I don't consider any of these blockers for a
>>> release
>>> >>> of
>>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an
>>> RC
>>> >>> on Feb
>>> >>>>>> 2. However if people are using a restarting scheduler
>>> (run_duration
>>> >>> is set)
>>> >>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791.
>>> So a
>>> >>>>>> workaround for this would be nice (we just updated dag_runs
>>> directly
>>> >>> in the
>>> >>>>>> database to say ‘finished’ before a certain date, but we are also
>>> not
>>> >>> using
>>> >>>>>> the run_duration).
>>> >>>>>>
>>> >>>>>> Bolke
>>> >>>>>>
>>> >>>>>>
>>> >>>>>>
>>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
>>> wrote:
>>> >>>>>>>
>>> >>>>>>> Will do. And thanks.
>>> >>>>>>>
>>> >>>>>>> Adding another issue:
>>> >>>>>>>
>>> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
>>> reason.
>>> >>>>>>> Need to investigate why.
>>> >>>>>>>
>>> >>>>>>> Related but not root cause:
>>> >>>>>>> * Logging is so chatty that it gets really hard to find the real
>>> >>> issue
>>> >>>>>>>
>>> >>>>>>> Bolke.
>>> >>>>>>>
>>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>>> >>> .INVALID>
>>> >>>>>> wrote:
>>> >>>>>>>>
>>> >>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully
>>> some
>>> >>>>>> others
>>> >>>>>>>> are too. Do you mind creating jiras for these since you have the
>>> >>> full
>>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it to
>>> >>> myself:
>>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>> >>>>>>>>
>>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
>>> bdbruin@gmail.com>
>>> >>>>>> wrote:
>>> >>>>>>>>
>>> >>>>>>>>> This is to report back on some of the (early) experiences we
>>> have
>>> >>> with
>>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>> >>>>>>>>>
>>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>> >>>>>> developers.
>>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
>>> report
>>> >>> a
>>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising
>>> (but not
>>> >>>>>>>>> reporting back errors)
>>> >>>>>>>>>
>>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>> >>>>>>>>> We run in a secure environment which requires this variable to
>>> be
>>> >>>>>>>>> whitelisted if it is modified (needs to be added to
>>> UPDATING.md)
>>> >>>>>>>>>
>>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>> >>>>>>>>> Log gets flooded without a suggestion what to do
>>> >>>>>>>>>
>>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
>>> seemed to
>>> >>>>>> have a
>>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>>> >>>>>> message
>>> >>>>>>>>> making it very slow
>>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’ states
>>> were
>>> >>>>>> not
>>> >>>>>>>>> being updated
>>> >>>>>>>>> - These dag_runs would never er be marked anything else than
>>> >>> running
>>> >>>>>> for
>>> >>>>>>>>> some reason
>>> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
>>> >>> certain
>>> >>>>>> date
>>> >>>>>>>>> to -> finished
>>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
>>> >>>>>> “finished/failed”
>>> >>>>>>>>>
>>> >>>>>>>>> 5. Our umask is set to 027
>>> >>>>>>>>>
>>> >>>>>>>>>
>>> >>>>>>>
>>> >>>>>>
>>> >>>>>>
>>> >>>>>
>>> >>>
>>> >>>
>>> >>
>>>
>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Can confirm it's a slow query on task_instance table. Still digging.
Unfortunately, the query is truncated in my UI right now:

SELECT task_instance.task_id AS task_instance_...

On Mon, Jan 23, 2017 at 1:56 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Digging. Might be a bit.
>
> On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com> wrote:
>
>> Slow query log? Db load?
>>
>> B.
>>
>> Verstuurd vanaf mijn iPad
>>
>> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <cr...@apache.org>
>> het volgende geschreven:
>> >
>> > Note: 6.5 million TIs in the task_instance table.
>> >
>> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <
>> criccomini@apache.org>
>> > wrote:
>> >
>> >> Hey Bolke,
>> >>
>> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all
>> free
>> >> as well.
>> >>
>> >> I am thinking that this is DB related, given that it's pausing when
>> >> executing an update. Was looking at the update_state method in
>> models.py,
>> >> which logs right before the 15s pause.
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
>> >> wrote:
>> >>
>> >>> Hi Chris,
>> >>>
>> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
>> executor
>> >>> is faster in checking the state than the TaskInstance is able to exit
>> >>> itself. No, I don’t consider it normal, but it was sometimes reported
>> (
>> >>> https://github.com/apache/incubator-airflow/pull/1821) though not
>> really
>> >>> replicable as of yet.
>> >>>
>> >>> Parsing seems exceptionally slow, it might be worth looking at it
>> with a
>> >>> debugger. Very faint guess might be that something with the
>> multiprocessing
>> >>> part could do something with memory that is costly, but then we need
>> to
>> >>> know more about what is running on the system. Never clue of system
>> metrics
>> >>> could be helpful here.
>> >>>
>> >>> Bolke
>> >>>
>> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>> >>> wrote:
>> >>>>
>> >>>> Also, seeing this in EVERY task that runs:
>> >>>>
>> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>> >>>> instance has been externally set to queued. Taking the poison pill.
>> So
>> >>>> long.
>> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
>> return
>> >>> code 0
>> >>>>
>> >>>>
>> >>>> All successful tasks are showing this at the end of their logs. Is
>> this
>> >>>> normal?
>> >>>>
>> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>> >>> criccomini@apache.org>
>> >>>> wrote:
>> >>>>
>> >>>>> Hey all,
>> >>>>>
>> >>>>> I've upgraded on production. Things seem to be working so far (only
>> >>> been
>> >>>>> an hour), but I am seeing this in the scheduler logs:
>> >>>>>
>> >>>>> File Path
>> >>> PID
>> >>>>> Runtime    Last Runtime    Last Run
>> >>>>> ------------------------------------------------------------------
>> >>> -----
>> >>>>> ---------  --------------  -------------------
>> >>>>> ...
>> >>>>> /etc/airflow/dags/dags/elt/el/db.py
>> >>> 24793
>> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
>> >>>>> ...
>> >>>>>
>> >>>>> It seems to be taking more than 15 minutes to parse this DAG. Any
>> idea
>> >>>>> what's causing this? Scheduler config:
>> >>>>>
>> >>>>> [scheduler]
>> >>>>> job_heartbeat_sec = 5
>> >>>>> scheduler_heartbeat_sec = 5
>> >>>>> max_threads = 2
>> >>>>> child_process_log_directory = /var/log/airflow/scheduler
>> >>>>>
>> >>>>> The db.py file, itself, doesn't interact with any outside systems,
>> so I
>> >>>>> would have expected this to not take so long. It does, however,
>> >>>>> programmatically generate many DAGs within the single .py file.
>> >>>>>
>> >>>>> A snippet of the scheduler log is here:
>> >>>>>
>> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> >>>>>
>> >>>>> Note how there are 10-15 second gaps occasionally. Any idea what's
>> >>> going
>> >>>>> on?
>> >>>>>
>> >>>>> Cheers,
>> >>>>> Chris
>> >>>>>
>> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
>> >>> wrote:
>> >>>>>
>> >>>>>> I created:
>> >>>>>>
>> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being checked,
>> but
>> >>>>>> not fixed
>> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t
>> get
>> >>> fixed
>> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>> >>> interval
>> >>>>>> was specified
>> >>>>>>
>> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a
>> PR.
>> >>>>>>
>> >>>>>> Please note that I don't consider any of these blockers for a
>> release
>> >>> of
>> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an
>> RC
>> >>> on Feb
>> >>>>>> 2. However if people are using a restarting scheduler (run_duration
>> >>> is set)
>> >>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So
>> a
>> >>>>>> workaround for this would be nice (we just updated dag_runs
>> directly
>> >>> in the
>> >>>>>> database to say ‘finished’ before a certain date, but we are also
>> not
>> >>> using
>> >>>>>> the run_duration).
>> >>>>>>
>> >>>>>> Bolke
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> >>>>>>>
>> >>>>>>> Will do. And thanks.
>> >>>>>>>
>> >>>>>>> Adding another issue:
>> >>>>>>>
>> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
>> reason.
>> >>>>>>> Need to investigate why.
>> >>>>>>>
>> >>>>>>> Related but not root cause:
>> >>>>>>> * Logging is so chatty that it gets really hard to find the real
>> >>> issue
>> >>>>>>>
>> >>>>>>> Bolke.
>> >>>>>>>
>> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>> >>> .INVALID>
>> >>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully
>> some
>> >>>>>> others
>> >>>>>>>> are too. Do you mind creating jiras for these since you have the
>> >>> full
>> >>>>>>>> context? I have created a JIRA for (1) and have assigned it to
>> >>> myself:
>> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>> >>>>>>>>
>> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
>> bdbruin@gmail.com>
>> >>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> This is to report back on some of the (early) experiences we
>> have
>> >>> with
>> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>> >>>>>>>>>
>> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>> >>>>>> developers.
>> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
>> report
>> >>> a
>> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising (but
>> not
>> >>>>>>>>> reporting back errors)
>> >>>>>>>>>
>> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> >>>>>>>>> We run in a secure environment which requires this variable to
>> be
>> >>>>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> >>>>>>>>>
>> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> >>>>>>>>> Log gets flooded without a suggestion what to do
>> >>>>>>>>>
>> >>>>>>>>> 4. At start up all running dag_runs are being checked, we
>> seemed to
>> >>>>>> have a
>> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>> >>>>>> message
>> >>>>>>>>> making it very slow
>> >>>>>>>>> - Checking would happen at every restart, but dag_runs’ states
>> were
>> >>>>>> not
>> >>>>>>>>> being updated
>> >>>>>>>>> - These dag_runs would never er be marked anything else than
>> >>> running
>> >>>>>> for
>> >>>>>>>>> some reason
>> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
>> >>> certain
>> >>>>>> date
>> >>>>>>>>> to -> finished
>> >>>>>>>>> -> need to investigate why dag_runs did not get marked
>> >>>>>> “finished/failed”
>> >>>>>>>>>
>> >>>>>>>>> 5. Our umask is set to 027
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>
>> >>>
>> >>>
>> >>
>>
>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Digging. Might be a bit.

On Mon, Jan 23, 2017 at 1:32 PM, Bolke de Bruin <bd...@gmail.com> wrote:

> Slow query log? Db load?
>
> B.
>
> Verstuurd vanaf mijn iPad
>
> > Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <cr...@apache.org>
> het volgende geschreven:
> >
> > Note: 6.5 million TIs in the task_instance table.
> >
> > On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <criccomini@apache.org
> >
> > wrote:
> >
> >> Hey Bolke,
> >>
> >> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all
> free
> >> as well.
> >>
> >> I am thinking that this is DB related, given that it's pausing when
> >> executing an update. Was looking at the update_state method in
> models.py,
> >> which logs right before the 15s pause.
> >>
> >> Cheers,
> >> Chris
> >>
> >> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
> >> wrote:
> >>
> >>> Hi Chris,
> >>>
> >>> What is the load on your machine? (CPU/IO/MEM) It seems that the
> executor
> >>> is faster in checking the state than the TaskInstance is able to exit
> >>> itself. No, I don’t consider it normal, but it was sometimes reported (
> >>> https://github.com/apache/incubator-airflow/pull/1821) though not
> really
> >>> replicable as of yet.
> >>>
> >>> Parsing seems exceptionally slow, it might be worth looking at it with
> a
> >>> debugger. Very faint guess might be that something with the
> multiprocessing
> >>> part could do something with memory that is costly, but then we need to
> >>> know more about what is running on the system. Never clue of system
> metrics
> >>> could be helpful here.
> >>>
> >>> Bolke
> >>>
> >>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
> >>> wrote:
> >>>>
> >>>> Also, seeing this in EVERY task that runs:
> >>>>
> >>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
> >>>> instance has been externally set to queued. Taking the poison pill. So
> >>>> long.
> >>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with
> return
> >>> code 0
> >>>>
> >>>>
> >>>> All successful tasks are showing this at the end of their logs. Is
> this
> >>>> normal?
> >>>>
> >>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
> >>> criccomini@apache.org>
> >>>> wrote:
> >>>>
> >>>>> Hey all,
> >>>>>
> >>>>> I've upgraded on production. Things seem to be working so far (only
> >>> been
> >>>>> an hour), but I am seeing this in the scheduler logs:
> >>>>>
> >>>>> File Path
> >>> PID
> >>>>> Runtime    Last Runtime    Last Run
> >>>>> ------------------------------------------------------------------
> >>> -----
> >>>>> ---------  --------------  -------------------
> >>>>> ...
> >>>>> /etc/airflow/dags/dags/elt/el/db.py
> >>> 24793
> >>>>> 43.41s     986.63s         2017-01-23T20:04:09
> >>>>> ...
> >>>>>
> >>>>> It seems to be taking more than 15 minutes to parse this DAG. Any
> idea
> >>>>> what's causing this? Scheduler config:
> >>>>>
> >>>>> [scheduler]
> >>>>> job_heartbeat_sec = 5
> >>>>> scheduler_heartbeat_sec = 5
> >>>>> max_threads = 2
> >>>>> child_process_log_directory = /var/log/airflow/scheduler
> >>>>>
> >>>>> The db.py file, itself, doesn't interact with any outside systems,
> so I
> >>>>> would have expected this to not take so long. It does, however,
> >>>>> programmatically generate many DAGs within the single .py file.
> >>>>>
> >>>>> A snippet of the scheduler log is here:
> >>>>>
> >>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
> >>>>>
> >>>>> Note how there are 10-15 second gaps occasionally. Any idea what's
> >>> going
> >>>>> on?
> >>>>>
> >>>>> Cheers,
> >>>>> Chris
> >>>>>
> >>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
> >>> wrote:
> >>>>>
> >>>>>> I created:
> >>>>>>
> >>>>>> - AIRFLOW-791: At start up all running dag_runs are being checked,
> but
> >>>>>> not fixed
> >>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
> >>> fixed
> >>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
> >>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
> >>> interval
> >>>>>> was specified
> >>>>>>
> >>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
> >>>>>>
> >>>>>> Please note that I don't consider any of these blockers for a
> release
> >>> of
> >>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
> >>> on Feb
> >>>>>> 2. However if people are using a restarting scheduler (run_duration
> >>> is set)
> >>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> >>>>>> workaround for this would be nice (we just updated dag_runs directly
> >>> in the
> >>>>>> database to say ‘finished’ before a certain date, but we are also
> not
> >>> using
> >>>>>> the run_duration).
> >>>>>>
> >>>>>> Bolke
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>>>>>>
> >>>>>>> Will do. And thanks.
> >>>>>>>
> >>>>>>> Adding another issue:
> >>>>>>>
> >>>>>>> * Some of our DAGs are not getting scheduled for some unknown
> reason.
> >>>>>>> Need to investigate why.
> >>>>>>>
> >>>>>>> Related but not root cause:
> >>>>>>> * Logging is so chatty that it gets really hard to find the real
> >>> issue
> >>>>>>>
> >>>>>>> Bolke.
> >>>>>>>
> >>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
> >>> .INVALID>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
> >>>>>> others
> >>>>>>>> are too. Do you mind creating jiras for these since you have the
> >>> full
> >>>>>>>> context? I have created a JIRA for (1) and have assigned it to
> >>> myself:
> >>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
> >>>>>>>>
> >>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <
> bdbruin@gmail.com>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> This is to report back on some of the (early) experiences we have
> >>> with
> >>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
> >>>>>>>>>
> >>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
> >>>>>> developers.
> >>>>>>>>> When a faulty dag is placed in the dags folder the UI would
> report
> >>> a
> >>>>>>>>> parsing error. Now it doesn’t due to the separate parising (but
> not
> >>>>>>>>> reporting back errors)
> >>>>>>>>>
> >>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >>>>>>>>> We run in a secure environment which requires this variable to be
> >>>>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
> >>>>>>>>>
> >>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> >>>>>>>>> Log gets flooded without a suggestion what to do
> >>>>>>>>>
> >>>>>>>>> 4. At start up all running dag_runs are being checked, we seemed
> to
> >>>>>> have a
> >>>>>>>>> lot of “left over” dag_runs (couple of thousand)
> >>>>>>>>> - Checking was logged to INFO -> requires a fsync for every log
> >>>>>> message
> >>>>>>>>> making it very slow
> >>>>>>>>> - Checking would happen at every restart, but dag_runs’ states
> were
> >>>>>> not
> >>>>>>>>> being updated
> >>>>>>>>> - These dag_runs would never er be marked anything else than
> >>> running
> >>>>>> for
> >>>>>>>>> some reason
> >>>>>>>>> -> Applied work around to update all dag_run in sql before a
> >>> certain
> >>>>>> date
> >>>>>>>>> to -> finished
> >>>>>>>>> -> need to investigate why dag_runs did not get marked
> >>>>>> “finished/failed”
> >>>>>>>>>
> >>>>>>>>> 5. Our umask is set to 027
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>
> >>>
> >>
>

Re: Experiences with 1.8.0

Posted by Bolke de Bruin <bd...@gmail.com>.
Slow query log? Db load?

B.

Verstuurd vanaf mijn iPad

> Op 23 jan. 2017 om 21:59 heeft Chris Riccomini <cr...@apache.org> het volgende geschreven:
> 
> Note: 6.5 million TIs in the task_instance table.
> 
> On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <cr...@apache.org>
> wrote:
> 
>> Hey Bolke,
>> 
>> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all free
>> as well.
>> 
>> I am thinking that this is DB related, given that it's pausing when
>> executing an update. Was looking at the update_state method in models.py,
>> which logs right before the 15s pause.
>> 
>> Cheers,
>> Chris
>> 
>> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> 
>>> Hi Chris,
>>> 
>>> What is the load on your machine? (CPU/IO/MEM) It seems that the executor
>>> is faster in checking the state than the TaskInstance is able to exit
>>> itself. No, I don’t consider it normal, but it was sometimes reported (
>>> https://github.com/apache/incubator-airflow/pull/1821) though not really
>>> replicable as of yet.
>>> 
>>> Parsing seems exceptionally slow, it might be worth looking at it with a
>>> debugger. Very faint guess might be that something with the multiprocessing
>>> part could do something with memory that is costly, but then we need to
>>> know more about what is running on the system. Never clue of system metrics
>>> could be helpful here.
>>> 
>>> Bolke
>>> 
>>>> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>>> wrote:
>>>> 
>>>> Also, seeing this in EVERY task that runs:
>>>> 
>>>> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>>>> instance has been externally set to queued. Taking the poison pill. So
>>>> long.
>>>> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return
>>> code 0
>>>> 
>>>> 
>>>> All successful tasks are showing this at the end of their logs. Is this
>>>> normal?
>>>> 
>>>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>>> criccomini@apache.org>
>>>> wrote:
>>>> 
>>>>> Hey all,
>>>>> 
>>>>> I've upgraded on production. Things seem to be working so far (only
>>> been
>>>>> an hour), but I am seeing this in the scheduler logs:
>>>>> 
>>>>> File Path
>>> PID
>>>>> Runtime    Last Runtime    Last Run
>>>>> ------------------------------------------------------------------
>>> -----
>>>>> ---------  --------------  -------------------
>>>>> ...
>>>>> /etc/airflow/dags/dags/elt/el/db.py
>>> 24793
>>>>> 43.41s     986.63s         2017-01-23T20:04:09
>>>>> ...
>>>>> 
>>>>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>>>>> what's causing this? Scheduler config:
>>>>> 
>>>>> [scheduler]
>>>>> job_heartbeat_sec = 5
>>>>> scheduler_heartbeat_sec = 5
>>>>> max_threads = 2
>>>>> child_process_log_directory = /var/log/airflow/scheduler
>>>>> 
>>>>> The db.py file, itself, doesn't interact with any outside systems, so I
>>>>> would have expected this to not take so long. It does, however,
>>>>> programmatically generate many DAGs within the single .py file.
>>>>> 
>>>>> A snippet of the scheduler log is here:
>>>>> 
>>>>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>>>>> 
>>>>> Note how there are 10-15 second gaps occasionally. Any idea what's
>>> going
>>>>> on?
>>>>> 
>>>>> Cheers,
>>>>> Chris
>>>>> 
>>>>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
>>> wrote:
>>>>> 
>>>>>> I created:
>>>>>> 
>>>>>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>>>>>> not fixed
>>>>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
>>> fixed
>>>>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>>>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>>> interval
>>>>>> was specified
>>>>>> 
>>>>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>>>>>> 
>>>>>> Please note that I don't consider any of these blockers for a release
>>> of
>>>>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
>>> on Feb
>>>>>> 2. However if people are using a restarting scheduler (run_duration
>>> is set)
>>>>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>>>>>> workaround for this would be nice (we just updated dag_runs directly
>>> in the
>>>>>> database to say ‘finished’ before a certain date, but we are also not
>>> using
>>>>>> the run_duration).
>>>>>> 
>>>>>> Bolke
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>>>>>>> 
>>>>>>> Will do. And thanks.
>>>>>>> 
>>>>>>> Adding another issue:
>>>>>>> 
>>>>>>> * Some of our DAGs are not getting scheduled for some unknown reason.
>>>>>>> Need to investigate why.
>>>>>>> 
>>>>>>> Related but not root cause:
>>>>>>> * Logging is so chatty that it gets really hard to find the real
>>> issue
>>>>>>> 
>>>>>>> Bolke.
>>>>>>> 
>>>>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>>> .INVALID>
>>>>>> wrote:
>>>>>>>> 
>>>>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>>>>>> others
>>>>>>>> are too. Do you mind creating jiras for these since you have the
>>> full
>>>>>>>> context? I have created a JIRA for (1) and have assigned it to
>>> myself:
>>>>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>>>>> 
>>>>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> This is to report back on some of the (early) experiences we have
>>> with
>>>>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>>>>>> 
>>>>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>>>>> developers.
>>>>>>>>> When a faulty dag is placed in the dags folder the UI would report
>>> a
>>>>>>>>> parsing error. Now it doesn’t due to the separate parising (but not
>>>>>>>>> reporting back errors)
>>>>>>>>> 
>>>>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>>>>>> We run in a secure environment which requires this variable to be
>>>>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>>>>>>>>> 
>>>>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>>>>>>>> Log gets flooded without a suggestion what to do
>>>>>>>>> 
>>>>>>>>> 4. At start up all running dag_runs are being checked, we seemed to
>>>>>> have a
>>>>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>>>>>> message
>>>>>>>>> making it very slow
>>>>>>>>> - Checking would happen at every restart, but dag_runs’ states were
>>>>>> not
>>>>>>>>> being updated
>>>>>>>>> - These dag_runs would never er be marked anything else than
>>> running
>>>>>> for
>>>>>>>>> some reason
>>>>>>>>> -> Applied work around to update all dag_run in sql before a
>>> certain
>>>>>> date
>>>>>>>>> to -> finished
>>>>>>>>> -> need to investigate why dag_runs did not get marked
>>>>>> “finished/failed”
>>>>>>>>> 
>>>>>>>>> 5. Our umask is set to 027
>>>>>>>>> 
>>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>> 
>>> 
>> 

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Note: 6.5 million TIs in the task_instance table.

On Mon, Jan 23, 2017 at 12:58 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey Bolke,
>
> Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all free
> as well.
>
> I am thinking that this is DB related, given that it's pausing when
> executing an update. Was looking at the update_state method in models.py,
> which logs right before the 15s pause.
>
> Cheers,
> Chris
>
> On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com>
> wrote:
>
>> Hi Chris,
>>
>> What is the load on your machine? (CPU/IO/MEM) It seems that the executor
>> is faster in checking the state than the TaskInstance is able to exit
>> itself. No, I don’t consider it normal, but it was sometimes reported (
>> https://github.com/apache/incubator-airflow/pull/1821) though not really
>> replicable as of yet.
>>
>> Parsing seems exceptionally slow, it might be worth looking at it with a
>> debugger. Very faint guess might be that something with the multiprocessing
>> part could do something with memory that is costly, but then we need to
>> know more about what is running on the system. Never clue of system metrics
>> could be helpful here.
>>
>> Bolke
>>
>> > On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org>
>> wrote:
>> >
>> > Also, seeing this in EVERY task that runs:
>> >
>> > [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
>> > instance has been externally set to queued. Taking the poison pill. So
>> > long.
>> > [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return
>> code 0
>> >
>> >
>> > All successful tasks are showing this at the end of their logs. Is this
>> > normal?
>> >
>> > On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <
>> criccomini@apache.org>
>> > wrote:
>> >
>> >> Hey all,
>> >>
>> >> I've upgraded on production. Things seem to be working so far (only
>> been
>> >> an hour), but I am seeing this in the scheduler logs:
>> >>
>> >> File Path
>>  PID
>> >> Runtime    Last Runtime    Last Run
>> >> ------------------------------------------------------------------
>> -----
>> >> ---------  --------------  -------------------
>> >> ...
>> >> /etc/airflow/dags/dags/elt/el/db.py
>>  24793
>> >> 43.41s     986.63s         2017-01-23T20:04:09
>> >> ...
>> >>
>> >> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>> >> what's causing this? Scheduler config:
>> >>
>> >> [scheduler]
>> >> job_heartbeat_sec = 5
>> >> scheduler_heartbeat_sec = 5
>> >> max_threads = 2
>> >> child_process_log_directory = /var/log/airflow/scheduler
>> >>
>> >> The db.py file, itself, doesn't interact with any outside systems, so I
>> >> would have expected this to not take so long. It does, however,
>> >> programmatically generate many DAGs within the single .py file.
>> >>
>> >> A snippet of the scheduler log is here:
>> >>
>> >> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> >>
>> >> Note how there are 10-15 second gaps occasionally. Any idea what's
>> going
>> >> on?
>> >>
>> >> Cheers,
>> >> Chris
>> >>
>> >> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> >>
>> >>> I created:
>> >>>
>> >>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>> >>> not fixed
>> >>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
>> fixed
>> >>> - AIRFLOW-788: Context unexpectedly added to hive conf
>> >>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>> interval
>> >>> was specified
>> >>>
>> >>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>> >>>
>> >>> Please note that I don't consider any of these blockers for a release
>> of
>> >>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
>> on Feb
>> >>> 2. However if people are using a restarting scheduler (run_duration
>> is set)
>> >>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>> >>> workaround for this would be nice (we just updated dag_runs directly
>> in the
>> >>> database to say ‘finished’ before a certain date, but we are also not
>> using
>> >>> the run_duration).
>> >>>
>> >>> Bolke
>> >>>
>> >>>
>> >>>
>> >>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>> >>>>
>> >>>> Will do. And thanks.
>> >>>>
>> >>>> Adding another issue:
>> >>>>
>> >>>> * Some of our DAGs are not getting scheduled for some unknown reason.
>> >>>> Need to investigate why.
>> >>>>
>> >>>> Related but not root cause:
>> >>>> * Logging is so chatty that it gets really hard to find the real
>> issue
>> >>>>
>> >>>> Bolke.
>> >>>>
>> >>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com
>> .INVALID>
>> >>> wrote:
>> >>>>>
>> >>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>> >>> others
>> >>>>> are too. Do you mind creating jiras for these since you have the
>> full
>> >>>>> context? I have created a JIRA for (1) and have assigned it to
>> myself:
>> >>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>> >>>>>
>> >>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>> >>> wrote:
>> >>>>>
>> >>>>>> This is to report back on some of the (early) experiences we have
>> with
>> >>>>>> Airflow 1.8.0 (beta 1 at the moment):
>> >>>>>>
>> >>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>> >>> developers.
>> >>>>>> When a faulty dag is placed in the dags folder the UI would report
>> a
>> >>>>>> parsing error. Now it doesn’t due to the separate parising (but not
>> >>>>>> reporting back errors)
>> >>>>>>
>> >>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> >>>>>> We run in a secure environment which requires this variable to be
>> >>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> >>>>>>
>> >>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> >>>>>> Log gets flooded without a suggestion what to do
>> >>>>>>
>> >>>>>> 4. At start up all running dag_runs are being checked, we seemed to
>> >>> have a
>> >>>>>> lot of “left over” dag_runs (couple of thousand)
>> >>>>>> - Checking was logged to INFO -> requires a fsync for every log
>> >>> message
>> >>>>>> making it very slow
>> >>>>>> - Checking would happen at every restart, but dag_runs’ states were
>> >>> not
>> >>>>>> being updated
>> >>>>>> - These dag_runs would never er be marked anything else than
>> running
>> >>> for
>> >>>>>> some reason
>> >>>>>> -> Applied work around to update all dag_run in sql before a
>> certain
>> >>> date
>> >>>>>> to -> finished
>> >>>>>> -> need to investigate why dag_runs did not get marked
>> >>> “finished/failed”
>> >>>>>>
>> >>>>>> 5. Our umask is set to 027
>> >>>>>>
>> >>>>>>
>> >>>>
>> >>>
>> >>>
>> >>
>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Hey Bolke,

Re: system usage, it's pretty quiet <5% CPU usage. Mem is almost all free
as well.

I am thinking that this is DB related, given that it's pausing when
executing an update. Was looking at the update_state method in models.py,
which logs right before the 15s pause.

Cheers,
Chris

On Mon, Jan 23, 2017 at 12:51 PM, Bolke de Bruin <bd...@gmail.com> wrote:

> Hi Chris,
>
> What is the load on your machine? (CPU/IO/MEM) It seems that the executor
> is faster in checking the state than the TaskInstance is able to exit
> itself. No, I don’t consider it normal, but it was sometimes reported (
> https://github.com/apache/incubator-airflow/pull/1821) though not really
> replicable as of yet.
>
> Parsing seems exceptionally slow, it might be worth looking at it with a
> debugger. Very faint guess might be that something with the multiprocessing
> part could do something with memory that is costly, but then we need to
> know more about what is running on the system. Never clue of system metrics
> could be helpful here.
>
> Bolke
>
> > On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org> wrote:
> >
> > Also, seeing this in EVERY task that runs:
> >
> > [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
> > instance has been externally set to queued. Taking the poison pill. So
> > long.
> > [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return
> code 0
> >
> >
> > All successful tasks are showing this at the end of their logs. Is this
> > normal?
> >
> > On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <criccomini@apache.org
> >
> > wrote:
> >
> >> Hey all,
> >>
> >> I've upgraded on production. Things seem to be working so far (only been
> >> an hour), but I am seeing this in the scheduler logs:
> >>
> >> File Path
>  PID
> >> Runtime    Last Runtime    Last Run
> >> ------------------------------------------------------------------
> -----
> >> ---------  --------------  -------------------
> >> ...
> >> /etc/airflow/dags/dags/elt/el/db.py
>  24793
> >> 43.41s     986.63s         2017-01-23T20:04:09
> >> ...
> >>
> >> It seems to be taking more than 15 minutes to parse this DAG. Any idea
> >> what's causing this? Scheduler config:
> >>
> >> [scheduler]
> >> job_heartbeat_sec = 5
> >> scheduler_heartbeat_sec = 5
> >> max_threads = 2
> >> child_process_log_directory = /var/log/airflow/scheduler
> >>
> >> The db.py file, itself, doesn't interact with any outside systems, so I
> >> would have expected this to not take so long. It does, however,
> >> programmatically generate many DAGs within the single .py file.
> >>
> >> A snippet of the scheduler log is here:
> >>
> >> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
> >>
> >> Note how there are 10-15 second gaps occasionally. Any idea what's going
> >> on?
> >>
> >> Cheers,
> >> Chris
> >>
> >> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>
> >>> I created:
> >>>
> >>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
> >>> not fixed
> >>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
> fixed
> >>> - AIRFLOW-788: Context unexpectedly added to hive conf
> >>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
> interval
> >>> was specified
> >>>
> >>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
> >>>
> >>> Please note that I don't consider any of these blockers for a release
> of
> >>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
> on Feb
> >>> 2. However if people are using a restarting scheduler (run_duration is
> set)
> >>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> >>> workaround for this would be nice (we just updated dag_runs directly
> in the
> >>> database to say ‘finished’ before a certain date, but we are also not
> using
> >>> the run_duration).
> >>>
> >>> Bolke
> >>>
> >>>
> >>>
> >>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> >>>>
> >>>> Will do. And thanks.
> >>>>
> >>>> Adding another issue:
> >>>>
> >>>> * Some of our DAGs are not getting scheduled for some unknown reason.
> >>>> Need to investigate why.
> >>>>
> >>>> Related but not root cause:
> >>>> * Logging is so chatty that it gets really hard to find the real issue
> >>>>
> >>>> Bolke.
> >>>>
> >>>>> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com.
> INVALID>
> >>> wrote:
> >>>>>
> >>>>> I'd be happy to lend a hand fixing these issues and hopefully some
> >>> others
> >>>>> are too. Do you mind creating jiras for these since you have the full
> >>>>> context? I have created a JIRA for (1) and have assigned it to
> myself:
> >>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
> >>>>>
> >>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> >>> wrote:
> >>>>>
> >>>>>> This is to report back on some of the (early) experiences we have
> with
> >>>>>> Airflow 1.8.0 (beta 1 at the moment):
> >>>>>>
> >>>>>> 1. The UI does not show faulty DAG, leading to confusion for
> >>> developers.
> >>>>>> When a faulty dag is placed in the dags folder the UI would report a
> >>>>>> parsing error. Now it doesn’t due to the separate parising (but not
> >>>>>> reporting back errors)
> >>>>>>
> >>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >>>>>> We run in a secure environment which requires this variable to be
> >>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
> >>>>>>
> >>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> >>>>>> Log gets flooded without a suggestion what to do
> >>>>>>
> >>>>>> 4. At start up all running dag_runs are being checked, we seemed to
> >>> have a
> >>>>>> lot of “left over” dag_runs (couple of thousand)
> >>>>>> - Checking was logged to INFO -> requires a fsync for every log
> >>> message
> >>>>>> making it very slow
> >>>>>> - Checking would happen at every restart, but dag_runs’ states were
> >>> not
> >>>>>> being updated
> >>>>>> - These dag_runs would never er be marked anything else than running
> >>> for
> >>>>>> some reason
> >>>>>> -> Applied work around to update all dag_run in sql before a certain
> >>> date
> >>>>>> to -> finished
> >>>>>> -> need to investigate why dag_runs did not get marked
> >>> “finished/failed”
> >>>>>>
> >>>>>> 5. Our umask is set to 027
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>>
> >>
>
>

Re: Experiences with 1.8.0

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

What is the load on your machine? (CPU/IO/MEM) It seems that the executor is faster in checking the state than the TaskInstance is able to exit itself. No, I don’t consider it normal, but it was sometimes reported (https://github.com/apache/incubator-airflow/pull/1821) though not really replicable as of yet. 

Parsing seems exceptionally slow, it might be worth looking at it with a debugger. Very faint guess might be that something with the multiprocessing part could do something with memory that is costly, but then we need to know more about what is running on the system. Never clue of system metrics could be helpful here.

Bolke

> On 23 Jan 2017, at 21:34, Chris Riccomini <cr...@apache.org> wrote:
> 
> Also, seeing this in EVERY task that runs:
> 
> [2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
> instance has been externally set to queued. Taking the poison pill. So
> long.
> [2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return code 0
> 
> 
> All successful tasks are showing this at the end of their logs. Is this
> normal?
> 
> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
> wrote:
> 
>> Hey all,
>> 
>> I've upgraded on production. Things seem to be working so far (only been
>> an hour), but I am seeing this in the scheduler logs:
>> 
>> File Path                                                             PID
>> Runtime    Last Runtime    Last Run
>> ------------------------------------------------------------------  -----
>> ---------  --------------  -------------------
>> ...
>> /etc/airflow/dags/dags/elt/el/db.py                                 24793
>> 43.41s     986.63s         2017-01-23T20:04:09
>> ...
>> 
>> It seems to be taking more than 15 minutes to parse this DAG. Any idea
>> what's causing this? Scheduler config:
>> 
>> [scheduler]
>> job_heartbeat_sec = 5
>> scheduler_heartbeat_sec = 5
>> max_threads = 2
>> child_process_log_directory = /var/log/airflow/scheduler
>> 
>> The db.py file, itself, doesn't interact with any outside systems, so I
>> would have expected this to not take so long. It does, however,
>> programmatically generate many DAGs within the single .py file.
>> 
>> A snippet of the scheduler log is here:
>> 
>> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> 
>> Note how there are 10-15 second gaps occasionally. Any idea what's going
>> on?
>> 
>> Cheers,
>> Chris
>> 
>> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>> 
>>> I created:
>>> 
>>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>>> not fixed
>>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
>>> - AIRFLOW-788: Context unexpectedly added to hive conf
>>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
>>> was specified
>>> 
>>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>>> 
>>> Please note that I don't consider any of these blockers for a release of
>>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb
>>> 2. However if people are using a restarting scheduler (run_duration is set)
>>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>>> workaround for this would be nice (we just updated dag_runs directly in the
>>> database to say ‘finished’ before a certain date, but we are also not using
>>> the run_duration).
>>> 
>>> Bolke
>>> 
>>> 
>>> 
>>>> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>>>> 
>>>> Will do. And thanks.
>>>> 
>>>> Adding another issue:
>>>> 
>>>> * Some of our DAGs are not getting scheduled for some unknown reason.
>>>> Need to investigate why.
>>>> 
>>>> Related but not root cause:
>>>> * Logging is so chatty that it gets really hard to find the real issue
>>>> 
>>>> Bolke.
>>>> 
>>>>> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
>>> wrote:
>>>>> 
>>>>> I'd be happy to lend a hand fixing these issues and hopefully some
>>> others
>>>>> are too. Do you mind creating jiras for these since you have the full
>>>>> context? I have created a JIRA for (1) and have assigned it to myself:
>>>>> https://issues.apache.org/jira/browse/AIRFLOW-780
>>>>> 
>>>>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>>> wrote:
>>>>> 
>>>>>> This is to report back on some of the (early) experiences we have with
>>>>>> Airflow 1.8.0 (beta 1 at the moment):
>>>>>> 
>>>>>> 1. The UI does not show faulty DAG, leading to confusion for
>>> developers.
>>>>>> When a faulty dag is placed in the dags folder the UI would report a
>>>>>> parsing error. Now it doesn’t due to the separate parising (but not
>>>>>> reporting back errors)
>>>>>> 
>>>>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>>>>> We run in a secure environment which requires this variable to be
>>>>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>>>>>> 
>>>>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>>>>> Log gets flooded without a suggestion what to do
>>>>>> 
>>>>>> 4. At start up all running dag_runs are being checked, we seemed to
>>> have a
>>>>>> lot of “left over” dag_runs (couple of thousand)
>>>>>> - Checking was logged to INFO -> requires a fsync for every log
>>> message
>>>>>> making it very slow
>>>>>> - Checking would happen at every restart, but dag_runs’ states were
>>> not
>>>>>> being updated
>>>>>> - These dag_runs would never er be marked anything else than running
>>> for
>>>>>> some reason
>>>>>> -> Applied work around to update all dag_run in sql before a certain
>>> date
>>>>>> to -> finished
>>>>>> -> need to investigate why dag_runs did not get marked
>>> “finished/failed”
>>>>>> 
>>>>>> 5. Our umask is set to 027
>>>>>> 
>>>>>> 
>>>> 
>>> 
>>> 
>> 


Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Also, seeing this in EVERY task that runs:

[2017-01-23 20:26:13,777] {jobs.py:2112} WARNING - State of this
instance has been externally set to queued. Taking the poison pill. So
long.
[2017-01-23 20:26:13,841] {jobs.py:2051} INFO - Task exited with return code 0


All successful tasks are showing this at the end of their logs. Is this
normal?

On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I've upgraded on production. Things seem to be working so far (only been
> an hour), but I am seeing this in the scheduler logs:
>
> File Path                                                             PID
>  Runtime    Last Runtime    Last Run
> ------------------------------------------------------------------  -----
>  ---------  --------------  -------------------
> ...
> /etc/airflow/dags/dags/elt/el/db.py                                 24793
>  43.41s     986.63s         2017-01-23T20:04:09
> ...
>
> It seems to be taking more than 15 minutes to parse this DAG. Any idea
> what's causing this? Scheduler config:
>
> [scheduler]
> job_heartbeat_sec = 5
> scheduler_heartbeat_sec = 5
> max_threads = 2
> child_process_log_directory = /var/log/airflow/scheduler
>
> The db.py file, itself, doesn't interact with any outside systems, so I
> would have expected this to not take so long. It does, however,
> programmatically generate many DAGs within the single .py file.
>
> A snippet of the scheduler log is here:
>
> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>
> Note how there are 10-15 second gaps occasionally. Any idea what's going
> on?
>
> Cheers,
> Chris
>
> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>
>> I created:
>>
>> - AIRFLOW-791: At start up all running dag_runs are being checked, but
>> not fixed
>> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
>> - AIRFLOW-788: Context unexpectedly added to hive conf
>> - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
>> was specified
>>
>> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>>
>> Please note that I don't consider any of these blockers for a release of
>> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb
>> 2. However if people are using a restarting scheduler (run_duration is set)
>> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>> workaround for this would be nice (we just updated dag_runs directly in the
>> database to say ‘finished’ before a certain date, but we are also not using
>> the run_duration).
>>
>> Bolke
>>
>>
>>
>> > On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>> >
>> > Will do. And thanks.
>> >
>> > Adding another issue:
>> >
>> > * Some of our DAGs are not getting scheduled for some unknown reason.
>> > Need to investigate why.
>> >
>> > Related but not root cause:
>> > * Logging is so chatty that it gets really hard to find the real issue
>> >
>> > Bolke.
>> >
>> >> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
>> wrote:
>> >>
>> >> I'd be happy to lend a hand fixing these issues and hopefully some
>> others
>> >> are too. Do you mind creating jiras for these since you have the full
>> >> context? I have created a JIRA for (1) and have assigned it to myself:
>> >> https://issues.apache.org/jira/browse/AIRFLOW-780
>> >>
>> >> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> >>
>> >>> This is to report back on some of the (early) experiences we have with
>> >>> Airflow 1.8.0 (beta 1 at the moment):
>> >>>
>> >>> 1. The UI does not show faulty DAG, leading to confusion for
>> developers.
>> >>> When a faulty dag is placed in the dags folder the UI would report a
>> >>> parsing error. Now it doesn’t due to the separate parising (but not
>> >>> reporting back errors)
>> >>>
>> >>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> >>> We run in a secure environment which requires this variable to be
>> >>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> >>>
>> >>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> >>> Log gets flooded without a suggestion what to do
>> >>>
>> >>> 4. At start up all running dag_runs are being checked, we seemed to
>> have a
>> >>> lot of “left over” dag_runs (couple of thousand)
>> >>> - Checking was logged to INFO -> requires a fsync for every log
>> message
>> >>> making it very slow
>> >>> - Checking would happen at every restart, but dag_runs’ states were
>> not
>> >>> being updated
>> >>> - These dag_runs would never er be marked anything else than running
>> for
>> >>> some reason
>> >>> -> Applied work around to update all dag_run in sql before a certain
>> date
>> >>> to -> finished
>> >>> -> need to investigate why dag_runs did not get marked
>> “finished/failed”
>> >>>
>> >>> 5. Our umask is set to 027
>> >>>
>> >>>
>> >
>>
>>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Oops, sorry again. I misread what you said (did I mention lack of sleep?)
The parsing of the db.py file takes 15 MINUTES (900+ seconds), as shown in
the `last runtime`. The log lines WITHIN the db.py.log file (from the
scheduler) show 15 second gaps.

On Mon, Jan 23, 2017 at 12:35 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Oops, yes, 15 seconds, sorry.  Operating without much sleep. :P
>
> On Mon, Jan 23, 2017 at 12:35 PM, Arthur Wiedmer <arthur.wiedmer@gmail.com
> > wrote:
>
>> Chris,
>>
>> Just double checking, you mean more than 15 seconds not 15 minutes, right?
>>
>> Best,
>> Arthur
>>
>> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
>> wrote:
>>
>> > Hey all,
>> >
>> > I've upgraded on production. Things seem to be working so far (only
>> been an
>> > hour), but I am seeing this in the scheduler logs:
>> >
>> > File Path
>>  PID
>> >  Runtime    Last Runtime    Last Run
>> > ------------------------------------------------------------------
>> -----
>> >  ---------  --------------  -------------------
>> > ...
>> > /etc/airflow/dags/dags/elt/el/db.py
>>  24793
>> >  43.41s     986.63s         2017-01-23T20:04:09
>> > ...
>> >
>> > It seems to be taking more than 15 minutes to parse this DAG. Any idea
>> > what's causing this? Scheduler config:
>> >
>> > [scheduler]
>> > job_heartbeat_sec = 5
>> > scheduler_heartbeat_sec = 5
>> > max_threads = 2
>> > child_process_log_directory = /var/log/airflow/scheduler
>> >
>> > The db.py file, itself, doesn't interact with any outside systems, so I
>> > would have expected this to not take so long. It does, however,
>> > programmatically generate many DAGs within the single .py file.
>> >
>> > A snippet of the scheduler log is here:
>> >
>> > https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>> >
>> > Note how there are 10-15 second gaps occasionally. Any idea what's going
>> > on?
>> >
>> > Cheers,
>> > Chris
>> >
>> > On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
>> wrote:
>> >
>> > > I created:
>> > >
>> > > - AIRFLOW-791: At start up all running dag_runs are being checked, but
>> > not
>> > > fixed
>> > > - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
>> > fixed
>> > > - AIRFLOW-788: Context unexpectedly added to hive conf
>> > > - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
>> interval
>> > > was specified
>> > >
>> > > I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>> > >
>> > > Please note that I don't consider any of these blockers for a release
>> of
>> > > 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC
>> on
>> > Feb
>> > > 2. However if people are using a restarting scheduler (run_duration is
>> > set)
>> > > and have a lot of running dag runs they won’t like AIRFLOW-791. So a
>> > > workaround for this would be nice (we just updated dag_runs directly
>> in
>> > the
>> > > database to say ‘finished’ before a certain date, but we are also not
>> > using
>> > > the run_duration).
>> > >
>> > > Bolke
>> > >
>> > >
>> > >
>> > > > On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
>> > > >
>> > > > Will do. And thanks.
>> > > >
>> > > > Adding another issue:
>> > > >
>> > > > * Some of our DAGs are not getting scheduled for some unknown
>> reason.
>> > > > Need to investigate why.
>> > > >
>> > > > Related but not root cause:
>> > > > * Logging is so chatty that it gets really hard to find the real
>> issue
>> > > >
>> > > > Bolke.
>> > > >
>> > > >> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com.
>> > INVALID>
>> > > wrote:
>> > > >>
>> > > >> I'd be happy to lend a hand fixing these issues and hopefully some
>> > > others
>> > > >> are too. Do you mind creating jiras for these since you have the
>> full
>> > > >> context? I have created a JIRA for (1) and have assigned it to
>> myself:
>> > > >> https://issues.apache.org/jira/browse/AIRFLOW-780
>> > > >>
>> > > >> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bdbruin@gmail.com
>> >
>> > > wrote:
>> > > >>
>> > > >>> This is to report back on some of the (early) experiences we have
>> > with
>> > > >>> Airflow 1.8.0 (beta 1 at the moment):
>> > > >>>
>> > > >>> 1. The UI does not show faulty DAG, leading to confusion for
>> > > developers.
>> > > >>> When a faulty dag is placed in the dags folder the UI would
>> report a
>> > > >>> parsing error. Now it doesn’t due to the separate parising (but
>> not
>> > > >>> reporting back errors)
>> > > >>>
>> > > >>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> > > >>> We run in a secure environment which requires this variable to be
>> > > >>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> > > >>>
>> > > >>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> > > >>> Log gets flooded without a suggestion what to do
>> > > >>>
>> > > >>> 4. At start up all running dag_runs are being checked, we seemed
>> to
>> > > have a
>> > > >>> lot of “left over” dag_runs (couple of thousand)
>> > > >>> - Checking was logged to INFO -> requires a fsync for every log
>> > message
>> > > >>> making it very slow
>> > > >>> - Checking would happen at every restart, but dag_runs’ states
>> were
>> > not
>> > > >>> being updated
>> > > >>> - These dag_runs would never er be marked anything else than
>> running
>> > > for
>> > > >>> some reason
>> > > >>> -> Applied work around to update all dag_run in sql before a
>> certain
>> > > date
>> > > >>> to -> finished
>> > > >>> -> need to investigate why dag_runs did not get marked
>> > > “finished/failed”
>> > > >>>
>> > > >>> 5. Our umask is set to 027
>> > > >>>
>> > > >>>
>> > > >
>> > >
>> > >
>> >
>>
>
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Oops, yes, 15 seconds, sorry.  Operating without much sleep. :P

On Mon, Jan 23, 2017 at 12:35 PM, Arthur Wiedmer <ar...@gmail.com>
wrote:

> Chris,
>
> Just double checking, you mean more than 15 seconds not 15 minutes, right?
>
> Best,
> Arthur
>
> On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
> > Hey all,
> >
> > I've upgraded on production. Things seem to be working so far (only been
> an
> > hour), but I am seeing this in the scheduler logs:
> >
> > File Path                                                             PID
> >  Runtime    Last Runtime    Last Run
> > ------------------------------------------------------------------
> -----
> >  ---------  --------------  -------------------
> > ...
> > /etc/airflow/dags/dags/elt/el/db.py
>  24793
> >  43.41s     986.63s         2017-01-23T20:04:09
> > ...
> >
> > It seems to be taking more than 15 minutes to parse this DAG. Any idea
> > what's causing this? Scheduler config:
> >
> > [scheduler]
> > job_heartbeat_sec = 5
> > scheduler_heartbeat_sec = 5
> > max_threads = 2
> > child_process_log_directory = /var/log/airflow/scheduler
> >
> > The db.py file, itself, doesn't interact with any outside systems, so I
> > would have expected this to not take so long. It does, however,
> > programmatically generate many DAGs within the single .py file.
> >
> > A snippet of the scheduler log is here:
> >
> > https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
> >
> > Note how there are 10-15 second gaps occasionally. Any idea what's going
> > on?
> >
> > Cheers,
> > Chris
> >
> > On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >
> > > I created:
> > >
> > > - AIRFLOW-791: At start up all running dag_runs are being checked, but
> > not
> > > fixed
> > > - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
> > fixed
> > > - AIRFLOW-788: Context unexpectedly added to hive conf
> > > - AIRFLOW-792: Allow fixing of schedule when wrong start_date /
> interval
> > > was specified
> > >
> > > I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
> > >
> > > Please note that I don't consider any of these blockers for a release
> of
> > > 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on
> > Feb
> > > 2. However if people are using a restarting scheduler (run_duration is
> > set)
> > > and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> > > workaround for this would be nice (we just updated dag_runs directly in
> > the
> > > database to say ‘finished’ before a certain date, but we are also not
> > using
> > > the run_duration).
> > >
> > > Bolke
> > >
> > >
> > >
> > > > On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> > > >
> > > > Will do. And thanks.
> > > >
> > > > Adding another issue:
> > > >
> > > > * Some of our DAGs are not getting scheduled for some unknown reason.
> > > > Need to investigate why.
> > > >
> > > > Related but not root cause:
> > > > * Logging is so chatty that it gets really hard to find the real
> issue
> > > >
> > > > Bolke.
> > > >
> > > >> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com.
> > INVALID>
> > > wrote:
> > > >>
> > > >> I'd be happy to lend a hand fixing these issues and hopefully some
> > > others
> > > >> are too. Do you mind creating jiras for these since you have the
> full
> > > >> context? I have created a JIRA for (1) and have assigned it to
> myself:
> > > >> https://issues.apache.org/jira/browse/AIRFLOW-780
> > > >>
> > > >> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> > > wrote:
> > > >>
> > > >>> This is to report back on some of the (early) experiences we have
> > with
> > > >>> Airflow 1.8.0 (beta 1 at the moment):
> > > >>>
> > > >>> 1. The UI does not show faulty DAG, leading to confusion for
> > > developers.
> > > >>> When a faulty dag is placed in the dags folder the UI would report
> a
> > > >>> parsing error. Now it doesn’t due to the separate parising (but not
> > > >>> reporting back errors)
> > > >>>
> > > >>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> > > >>> We run in a secure environment which requires this variable to be
> > > >>> whitelisted if it is modified (needs to be added to UPDATING.md)
> > > >>>
> > > >>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> > > >>> Log gets flooded without a suggestion what to do
> > > >>>
> > > >>> 4. At start up all running dag_runs are being checked, we seemed to
> > > have a
> > > >>> lot of “left over” dag_runs (couple of thousand)
> > > >>> - Checking was logged to INFO -> requires a fsync for every log
> > message
> > > >>> making it very slow
> > > >>> - Checking would happen at every restart, but dag_runs’ states were
> > not
> > > >>> being updated
> > > >>> - These dag_runs would never er be marked anything else than
> running
> > > for
> > > >>> some reason
> > > >>> -> Applied work around to update all dag_run in sql before a
> certain
> > > date
> > > >>> to -> finished
> > > >>> -> need to investigate why dag_runs did not get marked
> > > “finished/failed”
> > > >>>
> > > >>> 5. Our umask is set to 027
> > > >>>
> > > >>>
> > > >
> > >
> > >
> >
>

Re: Experiences with 1.8.0

Posted by Arthur Wiedmer <ar...@gmail.com>.
Chris,

Just double checking, you mean more than 15 seconds not 15 minutes, right?

Best,
Arthur

On Mon, Jan 23, 2017 at 12:27 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I've upgraded on production. Things seem to be working so far (only been an
> hour), but I am seeing this in the scheduler logs:
>
> File Path                                                             PID
>  Runtime    Last Runtime    Last Run
> ------------------------------------------------------------------  -----
>  ---------  --------------  -------------------
> ...
> /etc/airflow/dags/dags/elt/el/db.py                                 24793
>  43.41s     986.63s         2017-01-23T20:04:09
> ...
>
> It seems to be taking more than 15 minutes to parse this DAG. Any idea
> what's causing this? Scheduler config:
>
> [scheduler]
> job_heartbeat_sec = 5
> scheduler_heartbeat_sec = 5
> max_threads = 2
> child_process_log_directory = /var/log/airflow/scheduler
>
> The db.py file, itself, doesn't interact with any outside systems, so I
> would have expected this to not take so long. It does, however,
> programmatically generate many DAGs within the single .py file.
>
> A snippet of the scheduler log is here:
>
> https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65
>
> Note how there are 10-15 second gaps occasionally. Any idea what's going
> on?
>
> Cheers,
> Chris
>
> On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>
> > I created:
> >
> > - AIRFLOW-791: At start up all running dag_runs are being checked, but
> not
> > fixed
> > - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get
> fixed
> > - AIRFLOW-788: Context unexpectedly added to hive conf
> > - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
> > was specified
> >
> > I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
> >
> > Please note that I don't consider any of these blockers for a release of
> > 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on
> Feb
> > 2. However if people are using a restarting scheduler (run_duration is
> set)
> > and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> > workaround for this would be nice (we just updated dag_runs directly in
> the
> > database to say ‘finished’ before a certain date, but we are also not
> using
> > the run_duration).
> >
> > Bolke
> >
> >
> >
> > > On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> > >
> > > Will do. And thanks.
> > >
> > > Adding another issue:
> > >
> > > * Some of our DAGs are not getting scheduled for some unknown reason.
> > > Need to investigate why.
> > >
> > > Related but not root cause:
> > > * Logging is so chatty that it gets really hard to find the real issue
> > >
> > > Bolke.
> > >
> > >> On 20 Jan 2017, at 23:45, Dan Davydov <dan.davydov@airbnb.com.
> INVALID>
> > wrote:
> > >>
> > >> I'd be happy to lend a hand fixing these issues and hopefully some
> > others
> > >> are too. Do you mind creating jiras for these since you have the full
> > >> context? I have created a JIRA for (1) and have assigned it to myself:
> > >> https://issues.apache.org/jira/browse/AIRFLOW-780
> > >>
> > >> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> > wrote:
> > >>
> > >>> This is to report back on some of the (early) experiences we have
> with
> > >>> Airflow 1.8.0 (beta 1 at the moment):
> > >>>
> > >>> 1. The UI does not show faulty DAG, leading to confusion for
> > developers.
> > >>> When a faulty dag is placed in the dags folder the UI would report a
> > >>> parsing error. Now it doesn’t due to the separate parising (but not
> > >>> reporting back errors)
> > >>>
> > >>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> > >>> We run in a secure environment which requires this variable to be
> > >>> whitelisted if it is modified (needs to be added to UPDATING.md)
> > >>>
> > >>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> > >>> Log gets flooded without a suggestion what to do
> > >>>
> > >>> 4. At start up all running dag_runs are being checked, we seemed to
> > have a
> > >>> lot of “left over” dag_runs (couple of thousand)
> > >>> - Checking was logged to INFO -> requires a fsync for every log
> message
> > >>> making it very slow
> > >>> - Checking would happen at every restart, but dag_runs’ states were
> not
> > >>> being updated
> > >>> - These dag_runs would never er be marked anything else than running
> > for
> > >>> some reason
> > >>> -> Applied work around to update all dag_run in sql before a certain
> > date
> > >>> to -> finished
> > >>> -> need to investigate why dag_runs did not get marked
> > “finished/failed”
> > >>>
> > >>> 5. Our umask is set to 027
> > >>>
> > >>>
> > >
> >
> >
>

Re: Experiences with 1.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Hey all,

I've upgraded on production. Things seem to be working so far (only been an
hour), but I am seeing this in the scheduler logs:

File Path                                                             PID
 Runtime    Last Runtime    Last Run
------------------------------------------------------------------  -----
 ---------  --------------  -------------------
...
/etc/airflow/dags/dags/elt/el/db.py                                 24793
 43.41s     986.63s         2017-01-23T20:04:09
...

It seems to be taking more than 15 minutes to parse this DAG. Any idea
what's causing this? Scheduler config:

[scheduler]
job_heartbeat_sec = 5
scheduler_heartbeat_sec = 5
max_threads = 2
child_process_log_directory = /var/log/airflow/scheduler

The db.py file, itself, doesn't interact with any outside systems, so I
would have expected this to not take so long. It does, however,
programmatically generate many DAGs within the single .py file.

A snippet of the scheduler log is here:

https://gist.github.com/criccomini/a2b2762763c8ba65fefcdd669e8ffd65

Note how there are 10-15 second gaps occasionally. Any idea what's going on?

Cheers,
Chris

On Sun, Jan 22, 2017 at 4:42 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> I created:
>
> - AIRFLOW-791: At start up all running dag_runs are being checked, but not
> fixed
> - AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
> - AIRFLOW-788: Context unexpectedly added to hive conf
> - AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval
> was specified
>
> I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.
>
> Please note that I don't consider any of these blockers for a release of
> 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb
> 2. However if people are using a restarting scheduler (run_duration is set)
> and have a lot of running dag runs they won’t like AIRFLOW-791. So a
> workaround for this would be nice (we just updated dag_runs directly in the
> database to say ‘finished’ before a certain date, but we are also not using
> the run_duration).
>
> Bolke
>
>
>
> > On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> >
> > Will do. And thanks.
> >
> > Adding another issue:
> >
> > * Some of our DAGs are not getting scheduled for some unknown reason.
> > Need to investigate why.
> >
> > Related but not root cause:
> > * Logging is so chatty that it gets really hard to find the real issue
> >
> > Bolke.
> >
> >> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
> wrote:
> >>
> >> I'd be happy to lend a hand fixing these issues and hopefully some
> others
> >> are too. Do you mind creating jiras for these since you have the full
> >> context? I have created a JIRA for (1) and have assigned it to myself:
> >> https://issues.apache.org/jira/browse/AIRFLOW-780
> >>
> >> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >>
> >>> This is to report back on some of the (early) experiences we have with
> >>> Airflow 1.8.0 (beta 1 at the moment):
> >>>
> >>> 1. The UI does not show faulty DAG, leading to confusion for
> developers.
> >>> When a faulty dag is placed in the dags folder the UI would report a
> >>> parsing error. Now it doesn’t due to the separate parising (but not
> >>> reporting back errors)
> >>>
> >>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >>> We run in a secure environment which requires this variable to be
> >>> whitelisted if it is modified (needs to be added to UPDATING.md)
> >>>
> >>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> >>> Log gets flooded without a suggestion what to do
> >>>
> >>> 4. At start up all running dag_runs are being checked, we seemed to
> have a
> >>> lot of “left over” dag_runs (couple of thousand)
> >>> - Checking was logged to INFO -> requires a fsync for every log message
> >>> making it very slow
> >>> - Checking would happen at every restart, but dag_runs’ states were not
> >>> being updated
> >>> - These dag_runs would never er be marked anything else than running
> for
> >>> some reason
> >>> -> Applied work around to update all dag_run in sql before a certain
> date
> >>> to -> finished
> >>> -> need to investigate why dag_runs did not get marked
> “finished/failed”
> >>>
> >>> 5. Our umask is set to 027
> >>>
> >>>
> >
>
>

Re: Experiences with 1.8.0

Posted by Bolke de Bruin <bd...@gmail.com>.
I created:

- AIRFLOW-791: At start up all running dag_runs are being checked, but not fixed
- AIRFLOW-790: DagRuns do not exist for certain tasks, but don’t get fixed
- AIRFLOW-788: Context unexpectedly added to hive conf
- AIRFLOW-792: Allow fixing of schedule when wrong start_date / interval was specified

I created AIRFLOW-789 to update UPDATING.md, it is also out as a PR.

Please note that I don't consider any of these blockers for a release of 1.8.0 and can be fixed in 1.8.1 - so we are still on track for an RC on Feb 2. However if people are using a restarting scheduler (run_duration is set) and have a lot of running dag runs they won’t like AIRFLOW-791. So a workaround for this would be nice (we just updated dag_runs directly in the database to say ‘finished’ before a certain date, but we are also not using the run_duration).

Bolke



> On 20 Jan 2017, at 23:55, Bolke de Bruin <bd...@gmail.com> wrote:
> 
> Will do. And thanks.
> 
> Adding another issue: 
> 
> * Some of our DAGs are not getting scheduled for some unknown reason.
> Need to investigate why.
> 
> Related but not root cause:
> * Logging is so chatty that it gets really hard to find the real issue
> 
> Bolke.
> 
>> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID> wrote:
>> 
>> I'd be happy to lend a hand fixing these issues and hopefully some others
>> are too. Do you mind creating jiras for these since you have the full
>> context? I have created a JIRA for (1) and have assigned it to myself:
>> https://issues.apache.org/jira/browse/AIRFLOW-780
>> 
>> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com> wrote:
>> 
>>> This is to report back on some of the (early) experiences we have with
>>> Airflow 1.8.0 (beta 1 at the moment):
>>> 
>>> 1. The UI does not show faulty DAG, leading to confusion for developers.
>>> When a faulty dag is placed in the dags folder the UI would report a
>>> parsing error. Now it doesn’t due to the separate parising (but not
>>> reporting back errors)
>>> 
>>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>>> We run in a secure environment which requires this variable to be
>>> whitelisted if it is modified (needs to be added to UPDATING.md)
>>> 
>>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>>> Log gets flooded without a suggestion what to do
>>> 
>>> 4. At start up all running dag_runs are being checked, we seemed to have a
>>> lot of “left over” dag_runs (couple of thousand)
>>> - Checking was logged to INFO -> requires a fsync for every log message
>>> making it very slow
>>> - Checking would happen at every restart, but dag_runs’ states were not
>>> being updated
>>> - These dag_runs would never er be marked anything else than running for
>>> some reason
>>> -> Applied work around to update all dag_run in sql before a certain date
>>> to -> finished
>>> -> need to investigate why dag_runs did not get marked “finished/failed”
>>> 
>>> 5. Our umask is set to 027
>>> 
>>> 
> 


Re: Experiences with 1.8.0

Posted by Alex Van Boxel <al...@vanboxel.be>.
Bolke, I will tackle logging because I started integrating with Google
StackDriver logging. I have a separate branch, but this will be for after
1.8. It will be configurable and context aware and everyone will benefit
(not only stackdriver logging).

On Fri, Jan 20, 2017 at 11:55 PM Bolke de Bruin <bd...@gmail.com> wrote:

> Will do. And thanks.
>
> Adding another issue:
>
> * Some of our DAGs are not getting scheduled for some unknown reason.
> Need to investigate why.
>
> Related but not root cause:
> * Logging is so chatty that it gets really hard to find the real issue
>
> Bolke.
>
> > On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID>
> wrote:
> >
> > I'd be happy to lend a hand fixing these issues and hopefully some others
> > are too. Do you mind creating jiras for these since you have the full
> > context? I have created a JIRA for (1) and have assigned it to myself:
> > https://issues.apache.org/jira/browse/AIRFLOW-780
> >
> > On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com>
> wrote:
> >
> >> This is to report back on some of the (early) experiences we have with
> >> Airflow 1.8.0 (beta 1 at the moment):
> >>
> >> 1. The UI does not show faulty DAG, leading to confusion for developers.
> >> When a faulty dag is placed in the dags folder the UI would report a
> >> parsing error. Now it doesn’t due to the separate parising (but not
> >> reporting back errors)
> >>
> >> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> >> We run in a secure environment which requires this variable to be
> >> whitelisted if it is modified (needs to be added to UPDATING.md)
> >>
> >> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> >> Log gets flooded without a suggestion what to do
> >>
> >> 4. At start up all running dag_runs are being checked, we seemed to
> have a
> >> lot of “left over” dag_runs (couple of thousand)
> >> - Checking was logged to INFO -> requires a fsync for every log message
> >> making it very slow
> >> - Checking would happen at every restart, but dag_runs’ states were not
> >> being updated
> >> - These dag_runs would never er be marked anything else than running for
> >> some reason
> >> -> Applied work around to update all dag_run in sql before a certain
> date
> >> to -> finished
> >> -> need to investigate why dag_runs did not get marked “finished/failed”
> >>
> >> 5. Our umask is set to 027
> >>
> >>
>
> --
  _/
_/ Alex Van Boxel

Re: Experiences with 1.8.0

Posted by Bolke de Bruin <bd...@gmail.com>.
Will do. And thanks.

Adding another issue: 

* Some of our DAGs are not getting scheduled for some unknown reason.
Need to investigate why.

Related but not root cause:
* Logging is so chatty that it gets really hard to find the real issue

Bolke.

> On 20 Jan 2017, at 23:45, Dan Davydov <da...@airbnb.com.INVALID> wrote:
> 
> I'd be happy to lend a hand fixing these issues and hopefully some others
> are too. Do you mind creating jiras for these since you have the full
> context? I have created a JIRA for (1) and have assigned it to myself:
> https://issues.apache.org/jira/browse/AIRFLOW-780
> 
> On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com> wrote:
> 
>> This is to report back on some of the (early) experiences we have with
>> Airflow 1.8.0 (beta 1 at the moment):
>> 
>> 1. The UI does not show faulty DAG, leading to confusion for developers.
>> When a faulty dag is placed in the dags folder the UI would report a
>> parsing error. Now it doesn’t due to the separate parising (but not
>> reporting back errors)
>> 
>> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
>> We run in a secure environment which requires this variable to be
>> whitelisted if it is modified (needs to be added to UPDATING.md)
>> 
>> 3. DagRuns do not exist for certain tasks, but don’t get fixed
>> Log gets flooded without a suggestion what to do
>> 
>> 4. At start up all running dag_runs are being checked, we seemed to have a
>> lot of “left over” dag_runs (couple of thousand)
>> - Checking was logged to INFO -> requires a fsync for every log message
>> making it very slow
>> - Checking would happen at every restart, but dag_runs’ states were not
>> being updated
>> - These dag_runs would never er be marked anything else than running for
>> some reason
>> -> Applied work around to update all dag_run in sql before a certain date
>> to -> finished
>> -> need to investigate why dag_runs did not get marked “finished/failed”
>> 
>> 5. Our umask is set to 027
>> 
>> 


Re: Experiences with 1.8.0

Posted by Dan Davydov <da...@airbnb.com.INVALID>.
I'd be happy to lend a hand fixing these issues and hopefully some others
are too. Do you mind creating jiras for these since you have the full
context? I have created a JIRA for (1) and have assigned it to myself:
https://issues.apache.org/jira/browse/AIRFLOW-780

On Fri, Jan 20, 2017 at 1:01 AM, Bolke de Bruin <bd...@gmail.com> wrote:

> This is to report back on some of the (early) experiences we have with
> Airflow 1.8.0 (beta 1 at the moment):
>
> 1. The UI does not show faulty DAG, leading to confusion for developers.
> When a faulty dag is placed in the dags folder the UI would report a
> parsing error. Now it doesn’t due to the separate parising (but not
> reporting back errors)
>
> 2. The hive hook sets ‘airflow.ctx.dag_id’ in hive
> We run in a secure environment which requires this variable to be
> whitelisted if it is modified (needs to be added to UPDATING.md)
>
> 3. DagRuns do not exist for certain tasks, but don’t get fixed
> Log gets flooded without a suggestion what to do
>
> 4. At start up all running dag_runs are being checked, we seemed to have a
> lot of “left over” dag_runs (couple of thousand)
> - Checking was logged to INFO -> requires a fsync for every log message
> making it very slow
> - Checking would happen at every restart, but dag_runs’ states were not
> being updated
> - These dag_runs would never er be marked anything else than running for
> some reason
> -> Applied work around to update all dag_run in sql before a certain date
> to -> finished
> -> need to investigate why dag_runs did not get marked “finished/failed”
>
> 5. Our umask is set to 027
>
>