You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@airflow.apache.org by "Cieplucha, Michal" <mi...@intel.com> on 2018/04/04 12:08:30 UTC

slow scheduler

Hello all,

Our automation with airflow is getting bigger and bigger (airflow 1.8, ~150 DAGs, 3xinstances of scheduler) . Sometimes our users are triggering DAG runs based on some external events, so we exposed an API endpoint to run a DAG. Those DAGs that are run manually should give fast feedback to the user, but we see that it takes few minutes to schedule first task, and often next few minutes between tasks. So the most time is consumed between tasks, task durations are just some seconds. Does anybody have those issues? It looks like scheduler often have empty loops with logs like:
2018-04-04 12:05:45,004:DEBUG:airflow.jobs.SchedulerJob:[CT=None] Starting Loop...
2018-04-04 12:05:45,005:INFO:airflow.jobs.SchedulerJob:[CT=None] Heartbeating the process manager
2018-04-04 12:05:45,005:INFO:airflow.jobs.SchedulerJob:[CT=None] Heartbeating the executor
2018-04-04 12:05:45,005:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None] 44 running task instances
2018-04-04 12:05:45,005:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None] 0 in queue
2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None] 340 open slots
2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None] Calling the <class 'airflow.executors.celery_executor.CeleryExecutor'> sync method
2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None] Inquiring about 44 celery task(s)
2018-04-04 12:05:45,744:DEBUG:airflow.jobs.SchedulerJob:[CT=None] Ran scheduling loop in 0.74s
2018-04-04 12:05:45,745:DEBUG:airflow.jobs.SchedulerJob:[CT=None] Sleeping for 1.00s

Maybe we need to tune airflow settings?
We have up to 250 unacked messages on rabbit queue, which translates to number of running task instances, there is a lot going on in our airflow instance but apart from that scheduling issue everything looks fine (cpu/memory usage, etc).
Our general settings:
6x dockers with workers, parallelism is 384, dag concurrency 128 and celeryd_concurrency 64

Our scheduler config section:
job_heartbeat_sec = 5
scheduler_heartbeat_sec = 5
max_threads = 2


thanks
mC


I am an Intel employee. All comments and opinions are my own and do not represent the views of Intel.


--------------------------------------------------------------------

Intel Technology Poland sp. z o.o.
ul. Slowackiego 173 | 80-298 Gdansk | Sad Rejonowy Gdansk Polnoc | VII Wydzial Gospodarczy Krajowego Rejestru Sadowego - KRS 101882 | NIP 957-07-52-316 | Kapital zakladowy 200.000 PLN.

Ta wiadomosc wraz z zalacznikami jest przeznaczona dla okreslonego adresata i moze zawierac informacje poufne. W razie przypadkowego otrzymania tej wiadomosci, prosimy o powiadomienie nadawcy oraz trwale jej usuniecie; jakiekolwiek
przegladanie lub rozpowszechnianie jest zabronione.
This e-mail and any attachments may contain confidential material for the sole use of the intended recipient(s). If you are not the intended recipient, please contact the sender and delete all copies; any review or distribution by
others is strictly prohibited.

Re: slow scheduler

Posted by Maxime Beauchemin <ma...@gmail.com>.
As a batch scheduler Airflow doesn't currently guarantee super low latency.
The aim for the project has been to make it possible to do sub-minute
latency at scale, but it's common for this go up to a few minutes in larger
environments.

I'd recommend making it clear to your users what your Airflow environment
can currently guarantee, and that should inform how they define their
tasks. If latency is 1-2 minutes, it doesn't make sense to do a chain of
1-2 seconds tasks. Typically the duration of an Airflow task should be
counted in minutes, not seconds (there are exceptions though). In a way the
current latencies aren't bad as it weeds out things that probably shouldn't
be done in Airflow in the first place. Airflow is not Amazon Lambda.

Latency guarantees may get lower in the future as the community may tackle
"distributed scheduling", which wouldn't be hard to do. Essentially the
worker would evaluate and trigger child tasks as tasks succeed. Even when
that will be the case, I'd recommend avoiding small tasks as they would be
very overhead-heavy. You don't want the system to spend a too significant
portion of its effort on overhead (fetching and loading DAGs in memory,
maintaining/altering state in the database, ...).

Cranking up `max_threads` should help significantly (number of concurrent
DAGs that are getting scheduled), and I'd raise `job_heartbeat_sec`
and `scheduler_heartbeat_sec`
as the 5 seconds defaults are very low. They are set low to allow for
playing around with the examples with LocalExecutor. In production
environments you may want to raise this to 30 to 60 seconds. max_threads
depends on the box you're running the scheduler on, but you could run as
many threads as you have CPU cores on that box. Larger environments
typically "beef-up" their scheduler box and crank up the number of threads.

job_heartbeat_sec = 30
scheduler_heartbeat_sec = 30
max_threads = 32

We should clarify this in the documentation, and provide more guidance
around what a production `airflow.cfg` should look like.

Max

On Wed, Apr 4, 2018 at 5:08 AM, Cieplucha, Michal <
michal.cieplucha@intel.com> wrote:

> Hello all,
>
> Our automation with airflow is getting bigger and bigger (airflow 1.8,
> ~150 DAGs, 3xinstances of scheduler) . Sometimes our users are triggering
> DAG runs based on some external events, so we exposed an API endpoint to
> run a DAG. Those DAGs that are run manually should give fast feedback to
> the user, but we see that it takes few minutes to schedule first task, and
> often next few minutes between tasks. So the most time is consumed between
> tasks, task durations are just some seconds. Does anybody have those
> issues? It looks like scheduler often have empty loops with logs like:
> 2018-04-04 12:05:45,004:DEBUG:airflow.jobs.SchedulerJob:[CT=None]
> Starting Loop...
> 2018-04-04 12:05:45,005:INFO:airflow.jobs.SchedulerJob:[CT=None]
> Heartbeating the process manager
> 2018-04-04 12:05:45,005:INFO:airflow.jobs.SchedulerJob:[CT=None]
> Heartbeating the executor
> 2018-04-04 12:05:45,005:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None]
> 44 running task instances
> 2018-04-04 12:05:45,005:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None]
> 0 in queue
> 2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None]
> 340 open slots
> 2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None]
> Calling the <class 'airflow.executors.celery_executor.CeleryExecutor'>
> sync method
> 2018-04-04 12:05:45,006:DEBUG:airflow.executors.celery_executor.CeleryExecutor:[CT=None]
> Inquiring about 44 celery task(s)
> 2018-04-04 12:05:45,744:DEBUG:airflow.jobs.SchedulerJob:[CT=None] Ran
> scheduling loop in 0.74s
> 2018-04-04 12:05:45,745:DEBUG:airflow.jobs.SchedulerJob:[CT=None]
> Sleeping for 1.00s
>
> Maybe we need to tune airflow settings?
> We have up to 250 unacked messages on rabbit queue, which translates to
> number of running task instances, there is a lot going on in our airflow
> instance but apart from that scheduling issue everything looks fine
> (cpu/memory usage, etc).
> Our general settings:
> 6x dockers with workers, parallelism is 384, dag concurrency 128 and
> celeryd_concurrency 64
>
> Our scheduler config section:
> job_heartbeat_sec = 5
> scheduler_heartbeat_sec = 5
> max_threads = 2
>
>
> thanks
> mC
>
>
> I am an Intel employee. All comments and opinions are my own and do not
> represent the views of Intel.
>
>
> --------------------------------------------------------------------
>
> Intel Technology Poland sp. z o.o.
> ul. Slowackiego 173 | 80-298 Gdansk | Sad Rejonowy Gdansk Polnoc | VII
> Wydzial Gospodarczy Krajowego Rejestru Sadowego - KRS 101882 | NIP
> 957-07-52-316 | Kapital zakladowy 200.000 PLN.
>
> Ta wiadomosc wraz z zalacznikami jest przeznaczona dla okreslonego
> adresata i moze zawierac informacje poufne. W razie przypadkowego
> otrzymania tej wiadomosci, prosimy o powiadomienie nadawcy oraz trwale jej
> usuniecie; jakiekolwiek
> przegladanie lub rozpowszechnianie jest zabronione.
> This e-mail and any attachments may contain confidential material for the
> sole use of the intended recipient(s). If you are not the intended
> recipient, please contact the sender and delete all copies; any review or
> distribution by
> others is strictly prohibited.
>