We will need to come up with a plan soon (better DB indexes and/or the
ability to rotate out old task instances according to some policy). Nothing
concrete as of yet though.

On Tue, Mar 7, 2017 at 6:18 PM, Jason Chen <chingchien.c...@gmail.com>
wrote:

> Hi Dan,
>
>  Thanks so much. This is exactly what I am looking for.
>
> Is there a plan on the future airflow road map to clean this up from
> Airflow system level? Say, in airflow.cfg, a setting to clean up data older
> than specified time.
>
> Your solution is to run an airflow job to clean up the data. That's great.
> In a short term for us, I will be just running the SQL command directly
> from MySQL CLI and then setup an airflow job to do that periodically.
>
> Thanks.
> -Jason
>
> On Tue, Mar 7, 2017 at 5:47 PM, Dan Davydov <dan.davy...@airbnb.com.
> invalid>
> wrote:
>
> > FWIW we use the following DAG at Airbnb to reap the task instances table
> > (this is a stopgap):
> >
> > # DAG to delete old TIs so that UI operations on the webserver are fast.
> > This DAG is a
> > # stopgap, ideally we would make the UI not query all task instances and
> > add indexes to
> > # the task_instance table where appropriate to speed up the remaining
> > webserver table
> > # queries.
> > # Note that there is a slight risk that some of these deleted task
> > instances may break
> > # the depends_on_past dependency for the following tasks but this should
> > rarely happy
> > # and is easy to diagnose and fix.
> >
> > from datetime import datetime
> >
> > from airflow import DAG
> > from airflow.operators import MySqlOperator
> >
> > args = {
> >     'owner': 'xxx',
> >     'email': ['xxx'],
> >     'start_date': datetime(2017, 1, 30),
> >     'mysql_conn_id': 'airflow_db',
> > }
> >
> > dag = DAG(
> >     'airflow_old_task_instance_pruning',
> >     default_args=args,
> > )
> >
> > # TODO: TIs that have are successful without a start date will never be
> > # reaped because they have been mark-success'd in the UI. One fix for
> this
> > would be to
> > # make airflow set start_date when mark-success-ing.
> > sql = """\
> >     DELETE ti FROM task_instance ti
> >         LEFT OUTER JOIN dag_run dr
> >         ON ti.execution_date = dr.execution_date AND
> >            ti.dag_id = dr.dag_id
> >         WHERE ((ti.start_date <= DATE_SUB(NOW(), INTERVAL 30 DAY) AND
> >                     ti.state != "running") OR
> >                (ISNULL(ti.start_date) AND
> >                     ti.state = "failed")) AND
> >               (ISNULL(dr.id) OR dr.state != "running")
> > """
> > MySqlOperator(
> >     task_id='delete_old_tis',
> >     sql=sql,
> >     dag=dag,
> > )
> >
> >
> >
> > On Tue, Mar 7, 2017 at 5:39 PM, Jason Chen <chingchien.c...@gmail.com>
> > wrote:
> >
> > > Hi Bolke,
> > >
> > >  Thanks, but it looks you are actually talking about Harish's use case.
> > >
> > >  My use case is about 50 Dags (each one with about 2-3 tasks). I feel
> our
> > > run interval setting for the dags are too low (~15 mins). It may result
> > in
> > > high CPU of MySQL.
> > >
> > >  Meanwhile, I dig to MySQL and I noticed a frequently running SQL
> > statement
> > > as below. It's without proper index on column task_instance.state.
> > >
> > > Shouldn't it index "state", given that there could be million of rows
> in
> > > task_instance?
> > >
> > > SQL Statement:
> > > "SELECT task_instance.task_id AS task_instance_task_id,
> > > task_instance.dag_id AS task_instance_dag_id,.... FROM task_instance
> > WHERE
> > > task_instance.state = 'queued'"
> > >
> > > Also, is there a possibility to clean some "unneeded" entries in the
> > tables
> > > (say, task_instance) ?  I mean, for example, removing task states older
> > > than 6 months?
> > >
> > > Feedback are welcome.
> > >
> > > Thanks.
> > >
> > > -Jason
> > >
> > >
> > >
> > > On Tue, Mar 7, 2017 at 11:45 AM, Bolke de Bruin <bdbr...@gmail.com>
> > wrote:
> > >
> > > > Hi Jason
> > > >
> > > > I think you need to back it up with more numbers. You assume that a
> > load
> > > > of 100% is bad and also that 16GB of mem is a lot.
> > > >
> > > > 30x25 = 750 tasks per hour = 12,5 tasks per minute. For every task we
> > > > launch a couple of processes (at least 2) that do not share memory,
> > this
> > > is
> > > > to ensure tasks cannot hurt each other. Curl tasks are probably
> > launched
> > > by
> > > > using a BashOperator, which means another process. Curl is itself
> > another
> > > > process. So 4 processes per task, that cannot share memory. Curl can
> > > cache
> > > > memory itself as well. You probably have peak times and longer
> running
> > > > tasks so it is not evenly spread, then it starts adding up quickly?
> > > >
> > > > Bolke.
> > > >
> > > >
> > > > > On 7 Mar 2017, at 19:41, Jason Chen <chingchien.c...@gmail.com>
> > wrote:
> > > > >
> > > > > Hi Harish,
> > > > > Thanks for the fast response and feedback.
> > > > > Yeah, I want to see the fix or more discussion !
> > > > >
> > > > > BTW, I assume that, given your 30 dags, airflow runs fine after
> your
> > > > > increase of heartbeat ?
> > > > > The default is 5 secs.
> > > > >
> > > > >
> > > > > Thanks.
> > > > > Jason
> > > > >
> > > > >
> > > > > On Tue, Mar 7, 2017 at 10:24 AM, harish singh <
> > > harish.sing...@gmail.com>
> > > > > wrote:
> > > > >
> > > > >> I had seen a similar behavior, a year ago, when we were are < 5
> > Dags.
> > > > Even
> > > > >> then the cpu utilization was reaching 100%.
> > > > >> One way to deal with this is - You could play with "heatbeat"
> > numbers
> > > > (i.e
> > > > >> increase heartbeat).
> > > > >> But then you are introducing more delay to start jobs that are
> ready
> > > to
> > > > run
> > > > >> (ready to be queued -> queued -> run)
> > > > >>
> > > > >> Right now, we have more than 30 dags (each with ~ 20-25 tasks)
> that
> > > runs
> > > > >> every hour.
> > > > >> We are giving airflow about 5-6 cores (which still seems less for
> > > > airflow).
> > > > >> Also, for so many tasks every hour,  our mem consumption is over
> > 16G.
> > > > >> All our tasks are basically doing "curl". So 16G seems too high.
> > > > >>
> > > > >> Having said that, I remember reading somewhere that there was a
> fix
> > > > coming
> > > > >> for this.
> > > > >> If not, I would definitely want to see more discussion on this.
> > > > >>
> > > > >> Thanks for opening this. I would love to hear on how people are
> > > working
> > > > >> around this.
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >> On Tue, Mar 7, 2017 at 9:42 AM, Jason Chen <
> > chingchien.c...@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >>> Hi  team,
> > > > >>>
> > > > >>> We are using airflow v1.7.1.3 and schedule about 50 dags (each
> dags
> > > is
> > > > >>> about 10 to one hour intervals). It's with LocalExecutor.
> > > > >>>
> > > > >>> Recently, we noticed the RDS (MySQL 5.6.x with AWS) runs with
> ~100%
> > > > CPU.
> > > > >>> I am wondering if airflow scheduler and webserver can cause high
> > CPU
> > > > load
> > > > >>> of MySQL, given ~50 dags?
> > > > >>> I feel MySQL should be light load..
> > > > >>>
> > > > >>> Thanks.
> > > > >>> -Jason
> > > > >>>
> > > > >>
> > > >
> > > >
> > >
> >
>

Reply via email to