You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/12 13:33:57 UTC

[GitHub] [airflow] ashb commented on a change in pull request #11467: Add docs about Scheduler HA, how to use it and DB requirements

ashb commented on a change in pull request #11467:
URL: https://github.com/apache/airflow/pull/11467#discussion_r503299224



##########
File path: docs/scheduler.rst
##########
@@ -65,3 +65,72 @@ If you want to use 'external trigger' to run future-dated execution dates, set `
 This only has effect if your DAG has no ``schedule_interval``.
 If you keep default ``allow_trigger_in_future = False`` and try 'external trigger' to run future-dated execution dates,
 the scheduler won't execute it now but the scheduler will execute it in the future once the current date rolls over to the execution date.
+
+Running more than one scheduler
+-------------------------------
+
+Airflow 2.0 is the first release that officially supports running more one scheduler concurrently -- both for
+performance reasons and for resiliency.
+
+Overview
+""""""""
+
+The HA scheduler is written to take advantage of the existing metadata database. This was primarily done for
+operational simplicity: every component already has to speak to this DB, and by not using direct communication
+or consensus algorithm between schedulers (Raft, Paxos, etc.) nor another consensus tool (Apache Zookeeper, or
+Consul for instance) we have kept the "operational surface area" to a minimum.
+
+The scheduler now uses the serialized DAG representation to make it's scheduling decisions and the rough
+outline of the scheduling loop is:
+
+- Check for any DAGs needing a new DagRun, and create them
+- Examine a batch of DagRuns for schedulable TaskInstances or complete DagRuns
+- Select schedulable TaskInstances, and whilst respecting Pool limits and other concurrency limits, enqueue
+  them for execution
+
+This does however place some requirements on the Database.
+
+Database Requirements
+"""""""""""""""""""""
+
+The short version is that users of PostgreSQL 9.6+ or MySQL 8+ are all ready to go -- simple start running as
+many copies of the scheduler as you like -- there is no further set up or config options needed. If you are
+using a different database please read on
+
+To maintain performance and throughput there is one part of the scheduling loop that does a number of
+calculations in memory (because having to round-trip to the DB for each TaskInstance would be too slow) so we
+need to ensure that only a single scheduler is in this critical section at once - otherwise limits would not
+be correctly respected. To achieve this we use database row-level locks (using ``SELECT ... FOR UPDATE``).
+
+This critical section is where TaskInstances go from scheduled state and are enqueued to the executor, whilst
+ensuring the various concurrency and pool limits are respected. The critical section is obtained by asking for
+a row-level write lock on every row of the Pool table (roughly equivalent to ``SELECT * FROM slot_pool FOR
+UPDATE NOWAIT`` but the exact query is slightly different).
+
+The following databases are fully supported and provide an "optimal" experience:
+
+- PostgreSQL 9.6+
+- MySQL 8+
+
+MaraiDB will have slightly "degraded" performance, as it does not implement the ``SKIP LOCKED`` or ``NOWAIT`` SQL clauses --
+see `MDEV-13115 <https://jira.mariadb.org/browse/MDEV-13115>`_. Multiple schedulers will operate without this
+extension, but instead of skipping over the critical section and carrying on with other scheduling work
+(creating DagRuns, progressing TaskInstances to the scheduled state) they will wait for the critical section
+lock to be released.
+
+.. warning::
+
+  MySQL 5.x also does not support ``SKIP LOCKED`` or ``NOWAIT``, and additionally is more prone to deciding
+  queries are deadlocked, so running with a more than a single scheduler on MySQL 5.x is not supported or
+  recommended.
+
+.. note::
+
+  Microsoft SQLServer has not been tested with HA.
+
+Tuneables
+"""""""""
+
+- :ref:`config:scheduler__max_dagruns_to_create_per_loop`

Review comment:
       I'm almost tempted to move the docs out of the config and include them here, or expand on them here and just leave a short version in the config and point at this section from there.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org