airflow-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Chris Riccomini <criccom...@apache.org>
Subject Re: Experiences with 1.8.0
Date Mon, 23 Jan 2017 21:56:16 GMT
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/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 <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
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>
> >>>
> >>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message