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



##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -241,11 +243,51 @@ each parameter by following the links):
 * :ref:`config:scheduler__parsing_processes`
 * :ref:`config:scheduler__file_parsing_sort_mode`
 
+.. _best_practices/reducing_dag_complexity:
+
+Reducing DAG complexity
+^^^^^^^^^^^^^^^^^^^^^^^
+
+While Airflow is good in handling a lot of DAGs with a lot of task and 
dependencies between them, when you
+have many complex DAGs, their complexity might impact performance of 
scheduling. One of the ways to keep
+your Airflow instance performant and well utilized, you should strive to 
simplify and optimize your DAGs
+whenever possible - you have to remember that DAG parsing process and creation 
is just executing
+Python code and it's up to you to make it as performant as possible. There are 
no magic recipes for making
+your DAG "less complex" - since this is a Python code, it's the DAG writer who 
controls the complexity of
+their code.
+
+There are no "metrics" for DAG complexity, especially, there are no metrics 
that can tell you
+whether your DAG is "simple enough". However - as with any Python code you can 
definitely tell that
+your code is "simpler" or "faster" when you optimize it, the same can be said 
about DAG code. If you
+want to optimize your DAGs there are the following actions you can take:
+
+* Make your DAG load faster. This is a single improvement advice that might be 
implemented in various ways
+  but this is the one that has biggest impact on scheduler's performance. 
Whenever you have a chance to make
+  your DAG load faster - go for it, if your goal is to improve performance. 
See below
+  :ref:`best_practices/dag_loader_test` on how to asses your DAG loading time.
+
+* Make your DAG generate fewer tasks. Every task adds additional processing 
overhead for scheduling and
+  execution. If you can decrease the number of tasks that your DAG use, this 
will likely improve overall
+  scheduling and performance (however be aware that Airflow's flexibility 
comes from splitting the
+  work between multiple independent and sometimes parallel tasks and it makes 
it easier to reason
+  about the logic of your DAG when it is split to a number independent, 
standalone tasks. Also Airflow allows
+  to re-run only specific tasks when needed which might improve 
maintainability of the DAG - so you have to
+  strike the right balance between optimization, readability and 
maintainability which is best for your team.
+
+* Make smaller number of DAGs per file. While Airflow 2 is optimized for the 
case of having multiple DAGs
+  in one file, there are some parts of the system that make it sometimes less 
performant, or introduce more
+  delays than having those DAGs split among many files. Just the fact that one 
file can only be parsed by one
+  FileProcessor, makes it less scalable for example. If you have many DAGs 
generated from one file,
+  consider splitting them if you observe processing and scheduling delays.

Review comment:
       This makes zero difference to scheduling though, so we'll need to 
re-word this section.
   
   Since the scheduler operates soley on the serialized format, _once_ the dag 
is parsed once and in the serialized table the time to parse will have no 
impact on how fast the scheduler makes decisions about a DAG.
   
   The only time this would come back in to play is when executing the task 
(which is also only a problem if you are really trying to squeeze task start up 
time to a minimum.

##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -241,11 +243,51 @@ each parameter by following the links):
 * :ref:`config:scheduler__parsing_processes`
 * :ref:`config:scheduler__file_parsing_sort_mode`
 
+.. _best_practices/reducing_dag_complexity:
+
+Reducing DAG complexity
+^^^^^^^^^^^^^^^^^^^^^^^
+
+While Airflow is good in handling a lot of DAGs with a lot of task and 
dependencies between them, when you
+have many complex DAGs, their complexity might impact performance of 
scheduling. One of the ways to keep
+your Airflow instance performant and well utilized, you should strive to 
simplify and optimize your DAGs
+whenever possible - you have to remember that DAG parsing process and creation 
is just executing
+Python code and it's up to you to make it as performant as possible. There are 
no magic recipes for making
+your DAG "less complex" - since this is a Python code, it's the DAG writer who 
controls the complexity of
+their code.
+
+There are no "metrics" for DAG complexity, especially, there are no metrics 
that can tell you
+whether your DAG is "simple enough". However - as with any Python code you can 
definitely tell that
+your code is "simpler" or "faster" when you optimize it, the same can be said 
about DAG code. If you
+want to optimize your DAGs there are the following actions you can take:
+
+* Make your DAG load faster. This is a single improvement advice that might be 
implemented in various ways
+  but this is the one that has biggest impact on scheduler's performance. 
Whenever you have a chance to make
+  your DAG load faster - go for it, if your goal is to improve performance. 
See below
+  :ref:`best_practices/dag_loader_test` on how to asses your DAG loading time.
+
+* Make your DAG generate fewer tasks. Every task adds additional processing 
overhead for scheduling and
+  execution. If you can decrease the number of tasks that your DAG use, this 
will likely improve overall
+  scheduling and performance (however be aware that Airflow's flexibility 
comes from splitting the

Review comment:
       I'm not sure it does -- not to any really noticeable degree.
   
   The tree structure of the DAG has more impact than the number of the tasks I 
think (i.e. a linear chain of a->b->...z doesn't have to do all that much work 
as it only looks at the next task in the chain each time. I think?).
   
   But I'm not sure we should ever suggest that users make fewer tasks.

##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -255,9 +297,34 @@ No additional code needs to be written by the user to run 
this test.
 
 .. code-block:: bash
 
- python your-dag-file.py
+     python your-dag-file.py
+
+Running the above command without any error ensures your DAG does not contain 
any uninstalled dependency,
+syntax errors, etc. Make sure that you load your DAG in an environment that 
corresponds to your
+scheduler environment - with the same dependencies, environment variables, 
common code referred from the
+DAG.
+
+This is also a great way to check if your DAG loads faster after an 
optimization, if you want to attempt
+to optimize DAG loading time. Simply run the DAG and measure the time it 
takes, but again you have to
+make sure your DAG runs with the same dependencies, environment variables, 
common code.
+Make sure to run it several time in succession to account for caching effects. 
Compare the results
+before and after the optimization in order to assess the impact of the 
optimization.
+
+There are many ways to measure the time of processing, one of them in Linux 
environment is to
+use built-in ``time`` command
+
+.. code-block:: bash
+
+     time python your-dag-file.py
+
+Result:
+
+.. code-block:: text
+
+     python your-dag-file.py 0.05s user 0.02s system 1% cpu 1.033 total
 
-Running the above command without any error ensures your DAG does not contain 
any uninstalled dependency, syntax errors, etc.
+The important metrics is the "total time" - which tells you how long elapsed 
time it took
+to process the DAG.

Review comment:
       Might be worth adding a note that this is a slightly inflated time -- as 
_this_ command will have to `import airflow` etc, but when being loaded form 
within Airflow the `airflow` modules are already loaded pre-fork.

##########
File path: docs/apache-airflow/best-practices.rst
##########
@@ -241,11 +243,51 @@ each parameter by following the links):
 * :ref:`config:scheduler__parsing_processes`
 * :ref:`config:scheduler__file_parsing_sort_mode`
 
+.. _best_practices/reducing_dag_complexity:
+
+Reducing DAG complexity
+^^^^^^^^^^^^^^^^^^^^^^^
+
+While Airflow is good in handling a lot of DAGs with a lot of task and 
dependencies between them, when you
+have many complex DAGs, their complexity might impact performance of 
scheduling. One of the ways to keep
+your Airflow instance performant and well utilized, you should strive to 
simplify and optimize your DAGs
+whenever possible - you have to remember that DAG parsing process and creation 
is just executing
+Python code and it's up to you to make it as performant as possible. There are 
no magic recipes for making
+your DAG "less complex" - since this is a Python code, it's the DAG writer who 
controls the complexity of
+their code.
+
+There are no "metrics" for DAG complexity, especially, there are no metrics 
that can tell you
+whether your DAG is "simple enough". However - as with any Python code you can 
definitely tell that
+your code is "simpler" or "faster" when you optimize it, the same can be said 
about DAG code. If you
+want to optimize your DAGs there are the following actions you can take:
+
+* Make your DAG load faster. This is a single improvement advice that might be 
implemented in various ways
+  but this is the one that has biggest impact on scheduler's performance. 
Whenever you have a chance to make
+  your DAG load faster - go for it, if your goal is to improve performance. 
See below
+  :ref:`best_practices/dag_loader_test` on how to asses your DAG loading time.

Review comment:
       A good way of doing this when using Python callables is move imports 
from top level of file in to inside of the python callables.
   
   (Importing a big module such as numpy or pandas would do a surprising amount 
of disk io, and none of it should be needed at DAG definition/parse time)

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks
+   * How often the scheduler should perform cleanup and check for orphaned 
tasks/adopting them
+   * Whether scheduler uses row-level locking

Review comment:
       I'm not sure this should be mentioned, at least not without a massive 
warning -- turning it off means 1 scheduler only (or else incorrect behaviour!)

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -180,10 +338,43 @@ The following config settings can be used to control 
aspects of the Scheduler HA
   SchedulerJobs.
 
   This setting controls how a dead scheduler will be noticed and the tasks it
-  was "supervising" get picked up by another scheduler. (The tasks will stay
-  running, so there is no harm in not detecting this for a while.)
+  was "supervising" get picked up by another scheduler. The tasks will stay
+  running, so there is no harm in not detecting this for a while.
 
   When a SchedulerJob is detected as "dead" (as determined by
   :ref:`config:scheduler__scheduler_health_check_threshold`) any running or
   queued tasks that were launched by the dead process will be "adopted" and
   monitored by this scheduler instead.
+
+- :ref:`config:scheduler__dag_dir_list_interval`
+  How often (in seconds) to scan the DAGs directory for new files.
+
+- :ref:`config:scheduler__file_parsing_sort_mode`
+  The scheduler will list and sort the DAG files to decide the parsing order.
+
+- :ref:`config:scheduler__max_tis_per_query`
+  The batch size of queries in the scheduling main loop. If this is too high, 
SQL query
+  performance may be impacted by one or more of the following:
+
+  - reversion to full table scan - complexity of query predicate

Review comment:
       I don't think this is true -- it looks like it just applies a `LIMIT` to 
the query.
   
   (i.e. I can't see anywhere the gets TIs and then passes `n` of these to 
TI.filter_for_tis)

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -180,10 +338,43 @@ The following config settings can be used to control 
aspects of the Scheduler HA
   SchedulerJobs.
 
   This setting controls how a dead scheduler will be noticed and the tasks it
-  was "supervising" get picked up by another scheduler. (The tasks will stay
-  running, so there is no harm in not detecting this for a while.)
+  was "supervising" get picked up by another scheduler. The tasks will stay
+  running, so there is no harm in not detecting this for a while.
 
   When a SchedulerJob is detected as "dead" (as determined by
   :ref:`config:scheduler__scheduler_health_check_threshold`) any running or
   queued tasks that were launched by the dead process will be "adopted" and
   monitored by this scheduler instead.
+
+- :ref:`config:scheduler__dag_dir_list_interval`
+  How often (in seconds) to scan the DAGs directory for new files.
+
+- :ref:`config:scheduler__file_parsing_sort_mode`
+  The scheduler will list and sort the DAG files to decide the parsing order.
+
+- :ref:`config:scheduler__max_tis_per_query`
+  The batch size of queries in the scheduling main loop. If this is too high, 
SQL query
+  performance may be impacted by one or more of the following:
+
+  - reversion to full table scan - complexity of query predicate
+  - excessive locking
+
+  Additionally, you may hit the maximum allowable query length for your db.
+  Set this to 0 for no limit (not advised).
+
+- :ref:`config:scheduler__min_file_process_interval`
+  Number of seconds after which a DAG file is parsed. The DAG file is parsed 
every

Review comment:
       ```suggestion
     Number of seconds after which a DAG file is reparsed. The DAG file is 
parsed every
   ```

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -180,10 +338,43 @@ The following config settings can be used to control 
aspects of the Scheduler HA
   SchedulerJobs.
 
   This setting controls how a dead scheduler will be noticed and the tasks it
-  was "supervising" get picked up by another scheduler. (The tasks will stay
-  running, so there is no harm in not detecting this for a while.)
+  was "supervising" get picked up by another scheduler. The tasks will stay
+  running, so there is no harm in not detecting this for a while.
 
   When a SchedulerJob is detected as "dead" (as determined by
   :ref:`config:scheduler__scheduler_health_check_threshold`) any running or
   queued tasks that were launched by the dead process will be "adopted" and
   monitored by this scheduler instead.
+
+- :ref:`config:scheduler__dag_dir_list_interval`
+  How often (in seconds) to scan the DAGs directory for new files.
+
+- :ref:`config:scheduler__file_parsing_sort_mode`
+  The scheduler will list and sort the DAG files to decide the parsing order.
+
+- :ref:`config:scheduler__max_tis_per_query`
+  The batch size of queries in the scheduling main loop. If this is too high, 
SQL query
+  performance may be impacted by one or more of the following:
+
+  - reversion to full table scan - complexity of query predicate
+  - excessive locking
+
+  Additionally, you may hit the maximum allowable query length for your db.
+  Set this to 0 for no limit (not advised).
+
+- :ref:`config:scheduler__min_file_process_interval`
+  Number of seconds after which a DAG file is parsed. The DAG file is parsed 
every
+  min_file_process_interval number of seconds. Updates to DAGs are reflected 
after
+  this interval. Keeping this number low will increase CPU usage.
+
+- :ref:`config:scheduler__parsing_processes`
+  The scheduler can run multiple processes in parallel to parse DAGs. This 
defines
+  how many processes will run.
+
+- :ref:`config:scheduler__processor_poll_interval`
+  The number of seconds to wait between consecutive DAG file processing.

Review comment:
       This parameter is now badly named, as this is nothing to do with file 
processing anymore. Whoops.
   
   This controls how long the scheduler will sleep between loops, but _iff_ 
there was nothing to do in the loop. i.e. if it scheduled something then it 
will start the next loop iteration straight away.

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)

Review comment:
       ```suggestion
       * whether parsing your DAG file involves heavy processing at the top 
level (Hint! It should not. See :ref:`best-practices/top_level_code`)
   ```
   

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks
+   * How often the scheduler should perform cleanup and check for orphaned 
tasks/adopting them
+   * Whether scheduler uses row-level locking
+
+In order to perform fine-tuning, it's good to understand how Scheduler works 
under-the-hood.
+You can take a look at the ``Airflow Summit 2021``

Review comment:
       ```suggestion
   You can take a look at the Airflow Summit 2021 talk
   ```

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks
+   * How often the scheduler should perform cleanup and check for orphaned 
tasks/adopting them
+   * Whether scheduler uses row-level locking
+
+In order to perform fine-tuning, it's good to understand how Scheduler works 
under-the-hood.
+You can take a look at the ``Airflow Summit 2021``
+`Deep Dive into the Airflow Scheduler talk <https://youtu.be/DYC4-xElccE>`_ to 
perform the fine-tuning.
+
+How to approach Scheduler's fine-tuning
+"""""""""""""""""""""""""""""""""""""""
+
+Airflow gives you a lot of "knobs" to turn to fine tune the performance but 
it's a separate task,
+depending on your particular deployment, your DAG structure, hardware 
availability and expectations,
+to decide which knobs to turn to get best effect for you. Part of the job when 
managing the
+deployment is to decide what you are going to optimize for. Some users are ok 
with
+30 seconds delays of new DAG parsing, at the expense of lower CPU usage, 
whereas some other users
+expect the DAGs to be parsed almost instantly when they appear in the DAGs 
folder at the
+expense of higher CPU usage for example.
+
+Airflow gives you the flexibility to decide, but you should find out what 
aspect of performance is
+most important for you and decide which knobs you want to turn in which 
direction.
+
+Generally for fine-tuning, your approach should be the same as for any 
performance improvement and
+optimizations (we will not recommend any specific tools - just use the tools 
that you usually use
+to observe and monitor your systems):
+
+* its extremely important to monitor your system with the right set of tools 
that you usually use to
+  monitor your system. This document does not go into details of particular 
metrics and tools that you
+  can use, it just describes what kind of resources you should monitor, but 
you should follow your best
+  practices for monitoring to grab the right data.
+* decide which aspect of performance is most important for you (what you want 
to improve)
+* observe your system to see where your bottlenecks are: CPU, memory, I/O are 
the usual limiting factors
+* based on your expectations and observations - decide what is your next 
improvement and go back to
+  the observation of your performance, bottlenecks. Performance improvement is 
an iterative process.
+
+What resources might limit Scheduler's performance
+""""""""""""""""""""""""""""""""""""""""""""""""""
+
+There are several areas of resource usage that you should pay attention to:
+
+* FileSystem performance. Airflow Scheduler relies heavily on parsing 
(sometimes a lot) of Python
+  files, which are often located on a shared filesystem. Airflow Scheduler 
continuously reads and
+  re-parses those files. The same files have to be made available to workers, 
so often they are
+  stored in a distributed filesystem. You can use various filesystems for that 
purpose (NFS, CIFS, EFS,
+  GCS fuse, Azure File System are good examples). There are various parameters 
you can control for those
+  filesystems and fine-tune their performance, but this is beyond the scope of 
this document. You should
+  observe statistics and usage of your filesystem to determine if problems 
come from the filesystem
+  performance. For example there are anecdotal evidences that increasing IOPS 
(and paying more) for the
+  EFS performance, dramatically improves stability and speed of parsing 
Airflow DAGs when EFS is used.
+* Another solution to FileSystem performance, if it becomes your bottleneck, 
is to turn to alternative
+  mechanisms of distributing your DAGs. Embedding DAGs in your image and 
GitSync distribution have both
+  the property that the files are available locally for Scheduler and it does 
not have to use a
+  distributed filesystem to read the files, the files are available locally 
for the Scheduler and it is
+  usually as fast as it can be, especially if your machines use fast SSD disks 
for local storage. Those
+  distribution mechanisms have other characteristics that might make them not 
the best choice for you,
+  but if your problems with performance come from distributed filesystem 
performance, they might be the
+  best approach to follow.
+* Database connections and Database usage might become a problem as you want 
to increase performance and
+  process more things in parallel. Airflow is known from being 
"database-connection hungry" - the more DAGs
+  you have and the more you want to process in parallel, the more database 
connections will be opened.
+  This is generally not a problem for MySQL as its model of handling 
connections is thread-based, but this
+  might be a problem for Postgres, where connection handling is process-based. 
It is a general consensus
+  that if you have even medium size Postgres-based Airflow installation, the 
best solution is to use
+  `PGBouncer <https://www.pgbouncer.org/>`_ as a proxy to your database. The 
:doc:`helm-chart:index`
+  supports PGBouncer out-of-the-box. For MsSQL we have not yet worked out the 
best practices as support
+  for MsSQL is still experimental.
+* CPU usage is most important for FileProcessors - those are the processes 
that parse and execute
+  Python DAG files. Since Schedulers triggers such parsing continuously, when 
you have a lot of DAGs,
+  the processing might take a lot of CPU. You can mitigate it by decreasing the
+  :ref:`config:scheduler__min_file_process_interval`, but this is one of the 
mentioned trade-offs,
+  result of this is that changes to such files will be picked up slower and 
you will see delays between
+  submitting the files and getting them available in Airflow UI and executed 
by Scheduler. Optimizing
+  the way how your DAGs are built, avoiding external data sources is your best 
approach to improve CPU
+  usage. If you have more CPUs available, you can increase number of 
processing threads
+  :ref:`config:scheduler__parsing_processes`, Also Airflow Scheduler scales 
almost linearly with
+  several instances, so you can also add more Schedulers if your Scheduler's 
performance is CPU-bound.
+* Airflow might use quite significant amount of memory when you try to get 
more performance out of it.
+  Often more performance is achieved in Airflow by increasing number of 
processes handling the load,
+  and each process requires whole interpreter of Python loaded, a lot of 
classes imported, temporary

Review comment:
       Because of forking and copy-on-write, each process doesn't pay the full 
cost of this memory usage: it is shared between processes,
   
   i.e. one process might have 300Mb of active memory, but a second process 
doesn't need an extra 300Mb, it needs _almost_ nothing extra apart from the 
data it processes in memory.

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)

Review comment:
       ```suggestion
       * how large the DAG files are (remember dag parser needs to read and 
parse the file every n seconds)
   ```

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks

Review comment:
       I think we shouldn't mention this -- as modulo bugs that we've fixed 
this is _awlays_ going to be better as it distributes the work, and also 
performs scheduling on a smaller subset of the task graph!
   
   ```suggestion
   ```

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks
+   * How often the scheduler should perform cleanup and check for orphaned 
tasks/adopting them
+   * Whether scheduler uses row-level locking
+
+In order to perform fine-tuning, it's good to understand how Scheduler works 
under-the-hood.
+You can take a look at the ``Airflow Summit 2021``
+`Deep Dive into the Airflow Scheduler talk <https://youtu.be/DYC4-xElccE>`_ to 
perform the fine-tuning.
+
+How to approach Scheduler's fine-tuning
+"""""""""""""""""""""""""""""""""""""""
+
+Airflow gives you a lot of "knobs" to turn to fine tune the performance but 
it's a separate task,
+depending on your particular deployment, your DAG structure, hardware 
availability and expectations,
+to decide which knobs to turn to get best effect for you. Part of the job when 
managing the
+deployment is to decide what you are going to optimize for. Some users are ok 
with
+30 seconds delays of new DAG parsing, at the expense of lower CPU usage, 
whereas some other users
+expect the DAGs to be parsed almost instantly when they appear in the DAGs 
folder at the
+expense of higher CPU usage for example.
+
+Airflow gives you the flexibility to decide, but you should find out what 
aspect of performance is
+most important for you and decide which knobs you want to turn in which 
direction.
+
+Generally for fine-tuning, your approach should be the same as for any 
performance improvement and
+optimizations (we will not recommend any specific tools - just use the tools 
that you usually use
+to observe and monitor your systems):
+
+* its extremely important to monitor your system with the right set of tools 
that you usually use to
+  monitor your system. This document does not go into details of particular 
metrics and tools that you
+  can use, it just describes what kind of resources you should monitor, but 
you should follow your best
+  practices for monitoring to grab the right data.
+* decide which aspect of performance is most important for you (what you want 
to improve)
+* observe your system to see where your bottlenecks are: CPU, memory, I/O are 
the usual limiting factors
+* based on your expectations and observations - decide what is your next 
improvement and go back to
+  the observation of your performance, bottlenecks. Performance improvement is 
an iterative process.
+
+What resources might limit Scheduler's performance
+""""""""""""""""""""""""""""""""""""""""""""""""""
+
+There are several areas of resource usage that you should pay attention to:
+
+* FileSystem performance. Airflow Scheduler relies heavily on parsing 
(sometimes a lot) of Python

Review comment:
       This is less of a factor to scheduler overall, so I think we should move 
this further down the list.

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -138,18 +141,173 @@ The following databases are fully supported and provide 
an "optimal" experience:
 
   Microsoft SQLServer has not been tested with HA.
 
+
+Fine-tuning your Scheduler performance
+--------------------------------------
+
+What impacts scheduler's performance
+""""""""""""""""""""""""""""""""""""
+
+The Scheduler is responsible for two operations:
+
+* continuously parsing DAG files and synchronizing with the DAG in the database
+* continuously scheduling tasks for execution
+
+Those two tasks are executed in parallel by the scheduler and run 
independently of each other in
+different processes. In order to fine-tune your scheduler, you need to include 
a number of factors:
+
+* The kind of deployment you have
+    * what kind of filesystem you have to share the DAGs (impacts performance 
of continuously reading DAGs)
+    * how fast the filesystem is (in many cases of distributed cloud 
filesystem you can pay extra to get
+      more throughput/faster filesystem
+    * how much memory you have for your processing
+    * how much CPU you have available
+    * how much networking throughput you have available
+
+* The logic and definition of your DAG structure:
+    * how many DAG files you have
+    * how many DAGs you have in your files
+    * how large the DAG files are (remember scheduler needs to read and parse 
the file every n seconds)
+    * how complex they are (i.e. how fast they can be parsed, how many tasks 
and dependencies they have)
+    * whether parsing your DAGs involves heavy processing (Hint! It should 
not. See :doc:`/best-practices`)
+
+* The scheduler configuration
+   * How many schedulers you have
+   * How many parsing processes you have in your scheduler
+   * How much time scheduler waits between re-parsing of the same DAG (it 
happens continuously)
+   * How many task instances scheduler processes in one loop
+   * How many new DAG runs should be created/scheduled per loop
+   * Whether to execute "mini-scheduler" after completed task to speed up 
scheduling dependent tasks
+   * How often the scheduler should perform cleanup and check for orphaned 
tasks/adopting them
+   * Whether scheduler uses row-level locking
+
+In order to perform fine-tuning, it's good to understand how Scheduler works 
under-the-hood.
+You can take a look at the ``Airflow Summit 2021``
+`Deep Dive into the Airflow Scheduler talk <https://youtu.be/DYC4-xElccE>`_ to 
perform the fine-tuning.
+
+How to approach Scheduler's fine-tuning
+"""""""""""""""""""""""""""""""""""""""
+
+Airflow gives you a lot of "knobs" to turn to fine tune the performance but 
it's a separate task,
+depending on your particular deployment, your DAG structure, hardware 
availability and expectations,
+to decide which knobs to turn to get best effect for you. Part of the job when 
managing the
+deployment is to decide what you are going to optimize for. Some users are ok 
with
+30 seconds delays of new DAG parsing, at the expense of lower CPU usage, 
whereas some other users
+expect the DAGs to be parsed almost instantly when they appear in the DAGs 
folder at the
+expense of higher CPU usage for example.
+
+Airflow gives you the flexibility to decide, but you should find out what 
aspect of performance is
+most important for you and decide which knobs you want to turn in which 
direction.
+
+Generally for fine-tuning, your approach should be the same as for any 
performance improvement and
+optimizations (we will not recommend any specific tools - just use the tools 
that you usually use
+to observe and monitor your systems):
+
+* its extremely important to monitor your system with the right set of tools 
that you usually use to
+  monitor your system. This document does not go into details of particular 
metrics and tools that you
+  can use, it just describes what kind of resources you should monitor, but 
you should follow your best
+  practices for monitoring to grab the right data.
+* decide which aspect of performance is most important for you (what you want 
to improve)
+* observe your system to see where your bottlenecks are: CPU, memory, I/O are 
the usual limiting factors
+* based on your expectations and observations - decide what is your next 
improvement and go back to
+  the observation of your performance, bottlenecks. Performance improvement is 
an iterative process.
+
+What resources might limit Scheduler's performance
+""""""""""""""""""""""""""""""""""""""""""""""""""
+
+There are several areas of resource usage that you should pay attention to:
+
+* FileSystem performance. Airflow Scheduler relies heavily on parsing 
(sometimes a lot) of Python
+  files, which are often located on a shared filesystem. Airflow Scheduler 
continuously reads and
+  re-parses those files. The same files have to be made available to workers, 
so often they are
+  stored in a distributed filesystem. You can use various filesystems for that 
purpose (NFS, CIFS, EFS,
+  GCS fuse, Azure File System are good examples). There are various parameters 
you can control for those
+  filesystems and fine-tune their performance, but this is beyond the scope of 
this document. You should
+  observe statistics and usage of your filesystem to determine if problems 
come from the filesystem
+  performance. For example there are anecdotal evidences that increasing IOPS 
(and paying more) for the
+  EFS performance, dramatically improves stability and speed of parsing 
Airflow DAGs when EFS is used.
+* Another solution to FileSystem performance, if it becomes your bottleneck, 
is to turn to alternative
+  mechanisms of distributing your DAGs. Embedding DAGs in your image and 
GitSync distribution have both
+  the property that the files are available locally for Scheduler and it does 
not have to use a
+  distributed filesystem to read the files, the files are available locally 
for the Scheduler and it is
+  usually as fast as it can be, especially if your machines use fast SSD disks 
for local storage. Those
+  distribution mechanisms have other characteristics that might make them not 
the best choice for you,
+  but if your problems with performance come from distributed filesystem 
performance, they might be the
+  best approach to follow.
+* Database connections and Database usage might become a problem as you want 
to increase performance and
+  process more things in parallel. Airflow is known from being 
"database-connection hungry" - the more DAGs
+  you have and the more you want to process in parallel, the more database 
connections will be opened.
+  This is generally not a problem for MySQL as its model of handling 
connections is thread-based, but this
+  might be a problem for Postgres, where connection handling is process-based. 
It is a general consensus
+  that if you have even medium size Postgres-based Airflow installation, the 
best solution is to use
+  `PGBouncer <https://www.pgbouncer.org/>`_ as a proxy to your database. The 
:doc:`helm-chart:index`
+  supports PGBouncer out-of-the-box. For MsSQL we have not yet worked out the 
best practices as support
+  for MsSQL is still experimental.
+* CPU usage is most important for FileProcessors - those are the processes 
that parse and execute
+  Python DAG files. Since Schedulers triggers such parsing continuously, when 
you have a lot of DAGs,
+  the processing might take a lot of CPU. You can mitigate it by decreasing the
+  :ref:`config:scheduler__min_file_process_interval`, but this is one of the 
mentioned trade-offs,
+  result of this is that changes to such files will be picked up slower and 
you will see delays between
+  submitting the files and getting them available in Airflow UI and executed 
by Scheduler. Optimizing
+  the way how your DAGs are built, avoiding external data sources is your best 
approach to improve CPU
+  usage. If you have more CPUs available, you can increase number of 
processing threads
+  :ref:`config:scheduler__parsing_processes`, Also Airflow Scheduler scales 
almost linearly with
+  several instances, so you can also add more Schedulers if your Scheduler's 
performance is CPU-bound.
+* Airflow might use quite significant amount of memory when you try to get 
more performance out of it.
+  Often more performance is achieved in Airflow by increasing number of 
processes handling the load,
+  and each process requires whole interpreter of Python loaded, a lot of 
classes imported, temporary
+  in-memory storage. This can lead to memory pressure. You need to observe if 
your system is using
+  more memory than it has - which results with using swap disk, which 
dramatically decreases performance.
+  Note that Airflow Scheduler in versions prior to ``2.1.4`` generated a lot 
of ``Page Cache`` memory
+  used by log files (when the log files were not removed). This was generally 
harmless, as the memory
+  is just cache and could be reclaimed at any time by the system, however in 
version ``2.1.4`` and
+  beyond, writing logs will not generate excessive ``Page Cache`` memory. 
Regardless - make sure when you look
+  at memory usage, pay attention to the kind of memory you are observing. 
Usually you should look at
+  ``working memory``(names might vary depending on your deployment) rather 
than ``total memory used``.
+
+What can you do, to improve Scheduler's performance
+"""""""""""""""""""""""""""""""""""""""""""""""""""
+
+When you know what your resource usage is, the improvements that you can 
consider might be:
+
+* improve the logic, efficiency of parsing and reduce complexity of your 
top-level DAG Python code. It is
+  parsed continuously so optimizing that code might bring tremendous 
improvements, especially if you try
+  to reach out to some external databases etc. while parsing DAGs (this should 
be avoided at all cost).
+  The :ref:`best_practices/top_level_code` explains what are the best 
practices for writing your top-level
+  Python code. The :ref:`best_practices/reducing_dag_complexity` document 
provides some ares that you might
+  look at when you want to reduce complexity of your code.
+* improve utilization of your resources. This is when you have a free capacity 
in your system that
+  seems underutilized (again CPU, memory I/O, networking are the prime 
candidates) - you can take
+  actions like increasing number of schedulers, parsing processes or 
decreasing intervals for more
+  frequent actions might bring improvements in performance at the expense of 
higher utilization of those.
+* increase hardware capacity (for example if you see that CPU is limiting you 
or that I/O you use for
+  DAG filesystem is at its limits). Often the problem with scheduler 
performance is
+  simply because your system is not "capable" enough and this might be the 
only way. For example if
+  you see that you are using all CPU you have on machine, you might want to 
add another scheduler on
+  a new machine - in most cases, when you add 2nd or 3rd scheduler, the 
capacity of scheduling grows
+  linearly (unless the shared database or filesystem is a bottleneck).
+* experiment with different values for the "scheduler tunables". Often you 
might get better effects by
+  simply exchanging one performance aspect for another. For example if you 
want to decrease the
+  CPU usage, you might increase file processing interval (but the result will 
be that new DAGs will
+  appear with bigger delay). Usually performance tuning is the art of 
balancing different aspects.
+* sometimes you change scheduler behaviour slightly (for example change 
parsing sort order)
+  in order to get better fine-tuned results for your particular deployment.
+
+
 .. _scheduler:ha:tunables:
 
-Scheduler Tuneables
-"""""""""""""""""""
+Scheduler Configuration options
+"""""""""""""""""""""""""""""""
 
-The following config settings can be used to control aspects of the Scheduler 
HA loop.
+The following config settings can be used to control aspects of the Scheduler.
+However you can also look at other non-performance-related scheduler 
configuration parameters available at
+:doc:`../configurations-ref` in ``[scheduler]`` section.
 
 - :ref:`config:scheduler__max_dagruns_to_create_per_loop`
 
-  This changes the number of dags that are locked by each scheduler when
-  creating dag runs. One possible reason for setting this lower is if you
-  have huge dags and are running multiple schedules, you won't want one
+  This changes the number of DAGs that are locked by each scheduler when
+  creating DAG runs. One possible reason for setting this lower is if you
+  have huge DAGs and are running multiple schedules, you won't want one

Review comment:
       ```suggestion
     have huge DAGs (in the order of 10k+ tasks per DAG) and are running 
multiple schedulers, you won't want one
   ```

##########
File path: docs/apache-airflow/concepts/scheduler.rst
##########
@@ -180,10 +338,43 @@ The following config settings can be used to control 
aspects of the Scheduler HA
   SchedulerJobs.
 
   This setting controls how a dead scheduler will be noticed and the tasks it
-  was "supervising" get picked up by another scheduler. (The tasks will stay
-  running, so there is no harm in not detecting this for a while.)
+  was "supervising" get picked up by another scheduler. The tasks will stay
+  running, so there is no harm in not detecting this for a while.
 
   When a SchedulerJob is detected as "dead" (as determined by
   :ref:`config:scheduler__scheduler_health_check_threshold`) any running or
   queued tasks that were launched by the dead process will be "adopted" and
   monitored by this scheduler instead.
+
+- :ref:`config:scheduler__dag_dir_list_interval`
+  How often (in seconds) to scan the DAGs directory for new files.
+
+- :ref:`config:scheduler__file_parsing_sort_mode`
+  The scheduler will list and sort the DAG files to decide the parsing order.
+
+- :ref:`config:scheduler__max_tis_per_query`
+  The batch size of queries in the scheduling main loop. If this is too high, 
SQL query
+  performance may be impacted by one or more of the following:
+
+  - reversion to full table scan - complexity of query predicate
+  - excessive locking
+
+  Additionally, you may hit the maximum allowable query length for your db.
+  Set this to 0 for no limit (not advised).
+
+- :ref:`config:scheduler__min_file_process_interval`
+  Number of seconds after which a DAG file is parsed. The DAG file is parsed 
every
+  min_file_process_interval number of seconds. Updates to DAGs are reflected 
after
+  this interval. Keeping this number low will increase CPU usage.
+
+- :ref:`config:scheduler__parsing_processes`
+  The scheduler can run multiple processes in parallel to parse DAGs. This 
defines

Review comment:
       ```suggestion
     The scheduler can run multiple processes in parallel to parse DAG files. 
This defines
   ```




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to